You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by je...@apache.org on 2014/09/18 21:49:46 UTC
[01/25] TEZ-850. Recovery unit tests. (Jeff Zhang via hitesh)
Repository: tez
Updated Branches:
refs/heads/TEZ-8 d6589d3ac -> 625450cf1
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
index cd9a1e8..c5153b6 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
@@ -19,29 +19,28 @@
package org.apache.tez.dag.app.dag.impl;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.client.VertexStatus.State;
import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
import org.apache.tez.dag.api.oldrecords.TaskState;
import org.apache.tez.dag.app.AppContext;
@@ -58,110 +57,532 @@ import org.apache.tez.dag.app.dag.event.TaskEvent;
import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
import org.apache.tez.dag.app.dag.event.TaskEventType;
import org.apache.tez.dag.app.dag.event.VertexEventType;
-import org.apache.tez.dag.app.rm.container.AMContainer;
import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.history.events.TaskFinishedEvent;
import org.apache.tez.dag.history.events.TaskStartedEvent;
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.runtime.api.OutputCommitter;
+import org.apache.tez.runtime.api.OutputCommitterContext;
import org.junit.Before;
import org.junit.Test;
+import com.google.common.collect.Lists;
+
public class TestTaskRecovery {
- private static final Log LOG = LogFactory.getLog(TestTaskImpl.class);
+ private TaskImpl task;
+ private DrainDispatcher dispatcher;
- private int taskCounter = 0;
private int taskAttemptCounter = 0;
- private Configuration conf;
- private TaskAttemptListener taskAttemptListener;
- private TaskHeartbeatHandler taskHeartbeatHandler;
- private Credentials credentials;
- private Clock clock;
- private ApplicationId appId;
- private TezDAGID dagId;
- private TezVertexID vertexId;
+ private Configuration conf = new Configuration();
+ private AppContext mockAppContext;
+ private ApplicationId appId = ApplicationId.newInstance(
+ System.currentTimeMillis(), 1);
+ private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+ private TezVertexID vertexId = TezVertexID.getInstance(dagId, 1);
private Vertex vertex;
- private AppContext appContext;
- private Resource taskResource;
- private Map<String, LocalResource> localResources;
- private Map<String, String> environment;
- private String javaOpts;
- private boolean leafVertex;
- private ContainerContext containerContext;
- private ContainerId mockContainerId;
- private Container mockContainer;
- private AMContainer mockAMContainer;
- private NodeId mockNodeId;
-
- private TaskImpl task;
- private DrainDispatcher dispatcher;
+ private String vertexName = "v1";
+ private long taskScheduledTime = 100L;
+ private long taskStartTime = taskScheduledTime + 100L;
+ private long taskFinishTime = taskStartTime + 100L;
+ private TaskAttemptEventHandler taEventHandler =
+ new TaskAttemptEventHandler();
- class TaskEventHandler implements EventHandler<TaskEvent> {
+ private class TaskEventHandler implements EventHandler<TaskEvent> {
@Override
public void handle(TaskEvent event) {
task.handle(event);
}
}
- class TaskAttemptEventHandler implements EventHandler<TaskAttemptEvent> {
+ private class TaskAttemptEventHandler implements
+ EventHandler<TaskAttemptEvent> {
+
+ private List<TaskAttemptEvent> events = Lists.newArrayList();
+
@Override
public void handle(TaskAttemptEvent event) {
+ events.add(event);
((TaskAttemptImpl) task.getAttempt(event.getTaskAttemptID()))
.handle(event);
}
+
+ public List<TaskAttemptEvent> getEvents() {
+ return events;
+ }
}
- @Before
- public void setUp() {
- conf = new Configuration();
- taskAttemptListener = mock(TaskAttemptListener.class);
- taskHeartbeatHandler = mock(TaskHeartbeatHandler.class);
- credentials = new Credentials();
- clock = new SystemClock();
- appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
- dagId = TezDAGID.getInstance(appId, 1);
- vertexId = TezVertexID.getInstance(dagId, 1);
- vertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
- when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
- appContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
- mockContainerId = mock(ContainerId.class);
- mockContainer = mock(Container.class);
- mockAMContainer = mock(AMContainer.class);
- mockNodeId = mock(NodeId.class);
- when(mockContainer.getId()).thenReturn(mockContainerId);
- when(mockContainer.getNodeId()).thenReturn(mockNodeId);
- when(mockAMContainer.getContainer()).thenReturn(mockContainer);
- when(appContext.getAllContainers().get(mockContainerId)).thenReturn(
- mockAMContainer);
- when(appContext.getCurrentDAG().getVertex(any(TezVertexID.class)))
- .thenReturn(vertex);
- when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
+ private class TestOutputCommitter extends OutputCommitter {
+
+ boolean recoverySupported = false;
+ boolean throwExceptionWhenRecovery = false;
+
+ public TestOutputCommitter(OutputCommitterContext committerContext,
+ boolean recoverySupported, boolean throwExceptionWhenRecovery) {
+ super(committerContext);
+ this.recoverySupported = recoverySupported;
+ this.throwExceptionWhenRecovery = throwExceptionWhenRecovery;
+ }
+
+ @Override
+ public void recoverTask(int taskIndex, int previousDAGAttempt)
+ throws Exception {
+ if (throwExceptionWhenRecovery) {
+ throw new Exception("fail recovery Task");
+ }
+ }
+
+ @Override
+ public boolean isTaskRecoverySupported() {
+ return recoverySupported;
+ }
+
+ @Override
+ public void initialize() throws Exception {
+
+ }
+
+ @Override
+ public void setupOutput() throws Exception {
+
+ }
+
+ @Override
+ public void commitOutput() throws Exception {
+
+ }
+
+ @Override
+ public void abortOutput(State finalState) throws Exception {
- taskResource = Resource.newInstance(1024, 1);
- localResources = new HashMap<String, LocalResource>();
- environment = new HashMap<String, String>();
- javaOpts = "";
- leafVertex = false;
- containerContext =
- new ContainerContext(localResources, credentials, environment, javaOpts);
+ }
+
+ }
+ @Before
+ public void setUp() {
dispatcher = new DrainDispatcher();
dispatcher.register(DAGEventType.class, mock(EventHandler.class));
dispatcher.register(VertexEventType.class, mock(EventHandler.class));
dispatcher.register(TaskEventType.class, new TaskEventHandler());
- dispatcher.register(TaskAttemptEventType.class,
- new TaskAttemptEventHandler());
+ dispatcher.register(TaskAttemptEventType.class, taEventHandler);
dispatcher.init(new Configuration());
dispatcher.start();
+ vertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
+ when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
+
+ mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+ when(mockAppContext.getCurrentDAG().getVertex(any(TezVertexID.class)))
+ .thenReturn(vertex);
+
task =
- new TaskImpl(vertexId, 1, dispatcher.getEventHandler(), conf,
- taskAttemptListener, clock, taskHeartbeatHandler, appContext,
- leafVertex, taskResource, containerContext);
+ new TaskImpl(vertexId, 0, dispatcher.getEventHandler(),
+ new Configuration(), mock(TaskAttemptListener.class),
+ new SystemClock(), mock(TaskHeartbeatHandler.class),
+ mockAppContext, false, Resource.newInstance(1, 1),
+ mock(ContainerContext.class));
+
+ Map<String, OutputCommitter> committers =
+ new HashMap<String, OutputCommitter>();
+ committers.put("out1", new TestOutputCommitter(
+ mock(OutputCommitterContext.class), true, false));
+ when(task.getVertex().getOutputCommitters()).thenReturn(committers);
+ }
+
+ private void restoreFromTaskStartEvent() {
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskStartedEvent(task.getTaskId(),
+ vertexName, taskScheduledTime, taskStartTime));
+ assertEquals(TaskState.SCHEDULED, recoveredState);
+ assertEquals(0, task.finishedAttempts);
+ assertEquals(taskScheduledTime, task.scheduledTime);
+ assertEquals(0, task.getAttempts().size());
+ }
+
+ private void restoreFromFirstTaskAttemptStartEvent(TezTaskAttemptID taId) {
+ long taStartTime = taskStartTime + 100L;
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
+ taStartTime, mock(ContainerId.class), mock(NodeId.class), "", ""));
+ assertEquals(TaskState.RUNNING, recoveredState);
+ assertEquals(0, task.finishedAttempts);
+ assertEquals(taskScheduledTime, task.scheduledTime);
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(TaskAttemptStateInternal.NEW,
+ ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+ assertEquals(1, task.numberUncompletedAttempts);
+ }
+
+ /**
+ * New -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_New() {
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.NEW, task.getInternalState());
+ }
+
+ /**
+ * -> restoreFromTaskFinishEvent ( no TaskStartEvent )
+ */
+ @Test
+ public void testRecovery_NoStartEvent() {
+ try {
+ task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(), vertexName,
+ taskStartTime, taskFinishTime, null, TaskState.SUCCEEDED, "",
+ new TezCounters()));
+ fail("Should fail due to no TaskStartEvent before TaskFinishEvent");
+ } catch (Throwable e) {
+ assertTrue(e.getMessage().contains(
+ "Finished Event seen but"
+ + " no Started Event was encountered earlier"));
+ }
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_Started() {
+ restoreFromTaskStartEvent();
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+ // new task attempt is scheduled
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(0, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(null, task.successfulAttempt);
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * RecoverTranstion
+ */
+ @Test
+ public void testRecovery_OneTAStarted() {
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(0, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(null, task.successfulAttempt);
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_OneTAStarted_SUCCEEDED() {
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+
+ long taStartTime = taskStartTime + 100L;
+ long taFinishTime = taStartTime + 100L;
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+ new TezCounters()));
+ assertEquals(TaskState.SUCCEEDED, recoveredState);
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(taId, task.successfulAttempt);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(taId, task.successfulAttempt);
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * restoreFromTaskAttemptFinishedEvent (FAILED) -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_OneTAStarted_FAILED() {
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+
+ long taStartTime = taskStartTime + 100L;
+ long taFinishTime = taStartTime + 100L;
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ taStartTime, taFinishTime, TaskAttemptState.FAILED, "",
+ new TezCounters()));
+ assertEquals(TaskState.RUNNING, recoveredState);
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(1, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+ // new task attempt is scheduled
+ assertEquals(2, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(1, task.failedAttempts);
+ assertEquals(1, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * restoreFromTaskAttemptFinishedEvent (KILLED) -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_OneTAStarted_KILLED() {
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+
+ long taStartTime = taskStartTime + 100L;
+ long taFinishTime = taStartTime + 100L;
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ taStartTime, taFinishTime, TaskAttemptState.KILLED, "",
+ new TezCounters()));
+ assertEquals(TaskState.RUNNING, recoveredState);
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+ // new task attempt is scheduled
+ assertEquals(2, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(1, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) ->
+ * restoreFromTaskFinishedEvent -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_OneTAStarted_SUCCEEDED_Finished() {
+
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+
+ long taStartTime = taskStartTime + 100L;
+ long taFinishTime = taStartTime + 100L;
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+ new TezCounters()));
+ assertEquals(TaskState.SUCCEEDED, recoveredState);
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(taId, task.successfulAttempt);
+
+ recoveredState =
+ task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(),
+ vertexName, taskStartTime, taskFinishTime, taId,
+ TaskState.SUCCEEDED, "", new TezCounters()));
+ assertEquals(TaskState.SUCCEEDED, recoveredState);
+ assertEquals(taId, task.successfulAttempt);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(taId, task.successfulAttempt);
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_Commit_Failed_Recovery_Not_Supported() {
+ Map<String, OutputCommitter> committers =
+ new HashMap<String, OutputCommitter>();
+ committers.put("out1", new TestOutputCommitter(
+ mock(OutputCommitterContext.class), false, false));
+ when(task.getVertex().getOutputCommitters()).thenReturn(committers);
+
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+
+ // restoreFromTaskAttemptFinishedEvent (SUCCEEDED)
+ long taStartTime = taskStartTime + 100L;
+ long taFinishTime = taStartTime + 100L;
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+ new TezCounters()));
+ assertEquals(TaskState.SUCCEEDED, recoveredState);
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(taId, task.successfulAttempt);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+ // new task attempt is scheduled
+ assertEquals(2, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(1, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_Commit_Failed_recover_fail() {
+ Map<String, OutputCommitter> committers =
+ new HashMap<String, OutputCommitter>();
+ committers.put("out1", new TestOutputCommitter(
+ mock(OutputCommitterContext.class), true, true));
+ when(task.getVertex().getOutputCommitters()).thenReturn(committers);
+
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+
+ // restoreFromTaskAttemptFinishedEvent (SUCCEEDED)
+ long taStartTime = taskStartTime + 100L;
+ long taFinishTime = taStartTime + 100L;
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+ new TezCounters()));
+ assertEquals(TaskState.SUCCEEDED, recoveredState);
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(taId, task.successfulAttempt);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+ // new task attempt is scheduled
+ assertEquals(2, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(1, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+ }
+
+ @Test
+ public void testRecovery_WithDesired_SUCCEEDED() {
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+ task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.SUCCEEDED,
+ false));
+ assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+ // no TA_Recovery event sent
+ assertEquals(0, taEventHandler.getEvents().size());
+ }
+
+ @Test
+ public void testRecovery_WithDesired_FAILED() {
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+ task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.FAILED,
+ false));
+ assertEquals(TaskStateInternal.FAILED, task.getInternalState());
+ // no TA_Recovery event sent
+ assertEquals(0, taEventHandler.getEvents().size());
+ }
+
+ @Test
+ public void testRecovery_WithDesired_KILLED() {
+ restoreFromTaskStartEvent();
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ restoreFromFirstTaskAttemptStartEvent(taId);
+ task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.KILLED,
+ false));
+ assertEquals(TaskStateInternal.KILLED, task.getInternalState());
+ // no TA_Recovery event sent
+ assertEquals(0, taEventHandler.getEvents().size());
+
+ }
+
+ /**
+ * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+ * restoreFromTaskAttemptFinishedEvent (KILLED) -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_OneTAStarted_Killed() {
+ restoreFromTaskStartEvent();
+
+ long taStartTime = taskStartTime + 100L;
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ TaskState recoveredState =
+ task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
+ taStartTime, mock(ContainerId.class), mock(NodeId.class), "", ""));
+ assertEquals(TaskState.RUNNING, recoveredState);
+ assertEquals(TaskAttemptStateInternal.NEW,
+ ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(0, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(1, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+
+ long taFinishTime = taStartTime + 100L;
+ recoveredState =
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ taStartTime, taFinishTime, TaskAttemptState.KILLED, "",
+ new TezCounters()));
+ assertEquals(TaskState.RUNNING, recoveredState);
+ assertEquals(TaskAttemptStateInternal.NEW,
+ ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+ assertEquals(1, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(0, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
+
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
+ // wait for Task send TA_RECOVER to TA and TA complete the RecoverTransition
+ dispatcher.await();
+ assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+ assertEquals(TaskAttemptStateInternal.KILLED,
+ ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+ // new task attempt is scheduled
+ assertEquals(2, task.getAttempts().size());
+ assertEquals(1, task.finishedAttempts);
+ assertEquals(0, task.failedAttempts);
+ assertEquals(1, task.numberUncompletedAttempts);
+ assertEquals(null, task.successfulAttempt);
}
/**
@@ -170,24 +591,23 @@ public class TestTaskRecovery {
* schedule a new task attempt.
*/
@Test
- public void testTaskRecovery1() {
- TezTaskID lastTaskId = getNewTaskID();
- TezTaskID taskId = getNewTaskID();
+ public void testTaskRecovery_MultipleAttempts1() {
int maxFailedAttempts =
conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
- task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+ restoreFromTaskStartEvent();
+
for (int i = 0; i < maxFailedAttempts; ++i) {
- TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
- task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
- mockContainerId, mockNodeId, "", ""));
- task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
- TaskAttemptState.KILLED, "", null));
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
+ mock(ContainerId.class), mock(NodeId.class), "", ""));
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
+ 0, TaskAttemptState.KILLED, "", null));
}
assertEquals(maxFailedAttempts, task.getAttempts().size());
assertEquals(0, task.failedAttempts);
- task.handle(new TaskEventRecoverTask(lastTaskId));
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
// if the previous task attempt is killed, it should not been take into
// account when checking whether exceed the max attempts
assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
@@ -201,24 +621,23 @@ public class TestTaskRecovery {
* failed_attempt is exceeded.
*/
@Test
- public void testTaskRecovery2() {
- TezTaskID lastTaskId = getNewTaskID();
- TezTaskID taskId = getNewTaskID();
+ public void testTaskRecovery_MultipleAttempts2() {
int maxFailedAttempts =
conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
- task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+ restoreFromTaskStartEvent();
+
for (int i = 0; i < maxFailedAttempts; ++i) {
- TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
- task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
- mockContainerId, mockNodeId, "", ""));
- task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
- TaskAttemptState.FAILED, "", null));
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
+ mock(ContainerId.class), mock(NodeId.class), "", ""));
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
+ 0, TaskAttemptState.FAILED, "", null));
}
assertEquals(maxFailedAttempts, task.getAttempts().size());
assertEquals(maxFailedAttempts, task.failedAttempts);
- task.handle(new TaskEventRecoverTask(lastTaskId));
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
// it should transit to failed because of the failed task attempt in the
// last application attempt.
assertEquals(TaskStateInternal.FAILED, task.getInternalState());
@@ -232,34 +651,34 @@ public class TestTaskRecovery {
* state and new task attempt is scheduled.
*/
@Test
- public void testTaskRecovery3() throws InterruptedException {
- TezTaskID lastTaskId = getNewTaskID();
- TezTaskID taskId = getNewTaskID();
+ public void testTaskRecovery_MultipleAttempts3() throws InterruptedException {
int maxFailedAttempts =
conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
- task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+ restoreFromTaskStartEvent();
+
for (int i = 0; i < maxFailedAttempts - 1; ++i) {
- TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
- task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
- mockContainerId, mockNodeId, "", ""));
- task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
- TaskAttemptState.FAILED, "", null));
+ TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+ task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
+ mock(ContainerId.class), mock(NodeId.class), "", ""));
+ task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
+ 0, TaskAttemptState.FAILED, "", null));
}
assertEquals(maxFailedAttempts - 1, task.getAttempts().size());
assertEquals(maxFailedAttempts - 1, task.failedAttempts);
- TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(lastTaskId);
+ TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(task.getTaskId());
TaskState recoveredState =
task.restoreFromEvent(new TaskAttemptStartedEvent(newTaskAttemptId,
- "v1", 0, mockContainerId, mockNodeId, "", ""));
+ vertexName, 0, mock(ContainerId.class), mock(NodeId.class), "", ""));
+
assertEquals(TaskState.RUNNING, recoveredState);
assertEquals(TaskAttemptStateInternal.NEW,
((TaskAttemptImpl) task.getAttempt(newTaskAttemptId))
.getInternalState());
assertEquals(maxFailedAttempts, task.getAttempts().size());
- task.handle(new TaskEventRecoverTask(lastTaskId));
+ task.handle(new TaskEventRecoverTask(task.getTaskId()));
// wait until task attempt receive the Recover event from task
dispatcher.await();
@@ -273,12 +692,8 @@ public class TestTaskRecovery {
assertEquals(maxFailedAttempts + 1, task.getAttempts().size());
}
- private TezTaskID getNewTaskID() {
- TezTaskID taskID = TezTaskID.getInstance(vertexId, ++taskCounter);
- return taskID;
- }
-
private TezTaskAttemptID getNewTaskAttemptID(TezTaskID taskId) {
return TezTaskAttemptID.getInstance(taskId, taskAttemptCounter++);
}
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
new file mode 100644
index 0000000..e2f189c
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
@@ -0,0 +1,860 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.app.dag.impl;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+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.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataSourceType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeSchedulingType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.VertexState;
+import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventType;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
+import org.apache.tez.dag.app.dag.event.TaskEvent;
+import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
+import org.apache.tez.dag.app.dag.event.TaskEventType;
+import org.apache.tez.dag.app.dag.event.VertexEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
+import org.apache.tez.dag.app.dag.event.VertexEventType;
+import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter;
+import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexFinishedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexStartedEvent;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.runtime.api.OutputCommitter;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestVertexRecovery {
+
+ private static final Log LOG = LogFactory.getLog(TestVertexRecovery.class);
+
+ private DrainDispatcher dispatcher;
+
+ private AppContext mockAppContext;
+ private ApplicationId appId = ApplicationId.newInstance(
+ System.currentTimeMillis(), 1);
+ private DAGImpl dag;
+ private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+ private String user = "user";
+
+ private long initRequestedTime = 100L;
+ private long initedTime = initRequestedTime + 100L;
+
+ /*
+ * v1 v2 \ / v3
+ */
+ private DAGPlan createDAGPlan() {
+ DAGPlan dag =
+ DAGPlan
+ .newBuilder()
+ .setName("testverteximpl")
+ .addVertex(
+ VertexPlan
+ .newBuilder()
+ .setName("vertex1")
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host1")
+ .addRack("rack1").build())
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder().setNumTasks(1)
+ .setVirtualCores(4).setMemoryMb(1024)
+ .setJavaOpts("").setTaskModule("x1.y1").build())
+ .addOutEdgeId("e1")
+ .addOutputs(
+ DAGProtos.RootInputLeafOutputProto
+ .newBuilder()
+ .setIODescriptor(
+ TezEntityDescriptorProto.newBuilder()
+ .setClassName("output").build())
+ .setName("outputx")
+ .setControllerDescriptor(
+ TezEntityDescriptorProto
+ .newBuilder()
+ .setClassName(
+ CountingOutputCommitter.class.getName())))
+ .build())
+ .addVertex(
+ VertexPlan
+ .newBuilder()
+ .setName("vertex2")
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host2")
+ .addRack("rack2").build())
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder().setNumTasks(2)
+ .setVirtualCores(4).setMemoryMb(1024)
+ .setJavaOpts("").setTaskModule("x2.y2").build())
+ .addOutEdgeId("e2").build())
+ .addVertex(
+ VertexPlan
+ .newBuilder()
+ .setName("vertex3")
+ .setType(PlanVertexType.NORMAL)
+ .setProcessorDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName(
+ "x3.y3"))
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host3")
+ .addRack("rack3").build())
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder().setNumTasks(2)
+ .setVirtualCores(4).setMemoryMb(1024)
+ .setJavaOpts("foo").setTaskModule("x3.y3").build())
+ .addInEdgeId("e1")
+ .addInEdgeId("e2")
+ .addOutputs(
+ DAGProtos.RootInputLeafOutputProto
+ .newBuilder()
+ .setIODescriptor(
+ TezEntityDescriptorProto.newBuilder()
+ .setClassName("output").build())
+ .setName("outputx")
+ .setControllerDescriptor(
+ TezEntityDescriptorProto
+ .newBuilder()
+ .setClassName(
+ CountingOutputCommitter.class.getName())))
+ .build()
+
+ )
+
+ .addEdge(
+ EdgePlan
+ .newBuilder()
+ .setEdgeDestination(
+ TezEntityDescriptorProto.newBuilder().setClassName(
+ "i3_v1"))
+ .setInputVertexName("vertex1")
+ .setEdgeSource(
+ TezEntityDescriptorProto.newBuilder()
+ .setClassName("o1"))
+ .setOutputVertexName("vertex3")
+ .setDataMovementType(
+ PlanEdgeDataMovementType.SCATTER_GATHER).setId("e1")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build())
+ .addEdge(
+ EdgePlan
+ .newBuilder()
+ .setEdgeDestination(
+ TezEntityDescriptorProto.newBuilder().setClassName(
+ "i3_v2"))
+ .setInputVertexName("vertex2")
+ .setEdgeSource(
+ TezEntityDescriptorProto.newBuilder()
+ .setClassName("o2"))
+ .setOutputVertexName("vertex3")
+ .setDataMovementType(
+ PlanEdgeDataMovementType.SCATTER_GATHER).setId("e2")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()).build();
+
+ return dag;
+ }
+
+ class VertexEventHanlder implements EventHandler<VertexEvent> {
+
+ private List<VertexEvent> events = new ArrayList<VertexEvent>();
+
+ @Override
+ public void handle(VertexEvent event) {
+ events.add(event);
+ ((VertexImpl) dag.getVertex(event.getVertexId())).handle(event);
+ }
+
+ public List<VertexEvent> getEvents() {
+ return this.events;
+ }
+ }
+
+ class TaskEventHandler implements EventHandler<TaskEvent> {
+
+ private List<TaskEvent> events = new ArrayList<TaskEvent>();
+
+ @Override
+ public void handle(TaskEvent event) {
+ events.add(event);
+ ((TaskImpl) dag.getVertex(event.getTaskID().getVertexID()).getTask(
+ event.getTaskID())).handle(event);
+ }
+
+ public List<TaskEvent> getEvents() {
+ return events;
+ }
+ }
+
+ class TaskAttemptEventHandler implements EventHandler<TaskAttemptEvent> {
+
+ @Override
+ public void handle(TaskAttemptEvent event) {
+ // TezTaskID taskId = event.getTaskAttemptID().getTaskID();
+ // ((TaskAttemptImpl) vertex1.getTask(taskId).getAttempt(
+ // event.getTaskAttemptID())).handle(event);
+ }
+ }
+
+ private VertexEventHanlder vertexEventHandler;
+ private TaskEventHandler taskEventHandler;
+
+ @Before
+ public void setUp() throws IOException {
+
+ dispatcher = new DrainDispatcher();
+ dispatcher.register(DAGEventType.class, mock(EventHandler.class));
+ vertexEventHandler = new VertexEventHanlder();
+ dispatcher.register(VertexEventType.class, vertexEventHandler);
+ taskEventHandler = new TaskEventHandler();
+ dispatcher.register(TaskEventType.class, taskEventHandler);
+ dispatcher.register(TaskAttemptEventType.class,
+ new TaskAttemptEventHandler());
+ dispatcher.init(new Configuration());
+ dispatcher.start();
+
+ mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+
+ DAGPlan dagPlan = createDAGPlan();
+ dag =
+ new DAGImpl(dagId, new Configuration(), dagPlan,
+ dispatcher.getEventHandler(), mock(TaskAttemptListener.class),
+ new Credentials(), new SystemClock(), user,
+ mock(TaskHeartbeatHandler.class), mockAppContext);
+ when(mockAppContext.getCurrentDAG()).thenReturn(dag);
+
+ dag.handle(new DAGEvent(dagId, DAGEventType.DAG_INIT));
+ LOG.info("finish setUp");
+ }
+
+ /**
+ * vertex1(New) -> StartRecoveryTransition(SUCCEEDED)
+ */
+ @Test
+ public void testRecovery_Desired_SUCCEEDED() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+ "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+ assertEquals(VertexState.INITED, recoveredState);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.SUCCEEDED));
+ dispatcher.await();
+ assertEquals(VertexState.SUCCEEDED, vertex1.getState());
+ assertEquals(vertex1.numTasks, vertex1.succeededTaskCount);
+ assertEquals(vertex1.numTasks, vertex1.completedTaskCount);
+ // recover its task
+ assertTaskRecoveredEventSent(vertex1);
+
+ // vertex3 is still in NEW, when the desiredState is
+ // Completed State, each vertex recovery by itself, not depend on its parent
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ assertEquals(VertexState.NEW, vertex3.getState());
+ // no VertexEvent pass to downstream vertex
+ assertEquals(0, vertexEventHandler.getEvents().size());
+
+ }
+
+ /**
+ * vertex1(New) -> StartRecoveryTransition(FAILED)
+ */
+ @Test
+ public void testRecovery_Desired_FAILED() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+ "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+ assertEquals(VertexState.INITED, recoveredState);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.FAILED));
+ dispatcher.await();
+ assertEquals(VertexState.FAILED, vertex1.getState());
+ assertEquals(vertex1.numTasks, vertex1.failedTaskCount);
+ assertEquals(0, vertex1.completedTaskCount);
+ // recover its task
+ assertTaskRecoveredEventSent(vertex1);
+
+ // vertex3 is still in NEW, when the desiredState is
+ // Completed State, each vertex recovery by itself, not depend on its parent
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ assertEquals(VertexState.NEW, vertex3.getState());
+ // no VertexEvent pass to downstream vertex
+ assertEquals(0, vertexEventHandler.getEvents().size());
+ }
+
+ /**
+ * vertex1(New) -> StartRecoveryTransition(KILLED)
+ */
+ @Test
+ public void testRecovery_Desired_KILLED() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+ "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+ assertEquals(VertexState.INITED, recoveredState);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.KILLED));
+ dispatcher.await();
+ assertEquals(VertexState.KILLED, vertex1.getState());
+ assertEquals(vertex1.numTasks, vertex1.killedTaskCount);
+ assertEquals(0, vertex1.completedTaskCount);
+ // recover its task
+ assertTaskRecoveredEventSent(vertex1);
+
+ // vertex3 is still in NEW, when the desiredState is
+ // Completed State, each vertex recovery by itself, not depend on its parent
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ assertEquals(VertexState.NEW, vertex3.getState());
+ // no VertexEvent pass to downstream vertex
+ assertEquals(0, vertexEventHandler.getEvents().size());
+ }
+
+ /**
+ * vertex1(New) -> StartRecoveryTransition(ERROR)
+ */
+ @Test
+ public void testRecovery_Desired_ERROR() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+ "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+ assertEquals(VertexState.INITED, recoveredState);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.ERROR));
+ dispatcher.await();
+ assertEquals(VertexState.ERROR, vertex1.getState());
+ assertEquals(vertex1.numTasks, vertex1.failedTaskCount);
+ assertEquals(0, vertex1.completedTaskCount);
+ // recover its task
+ assertTaskRecoveredEventSent(vertex1);
+
+ // vertex3 is still in NEW, when the desiredState is
+ // Completed State, each vertex recovery by itself, not depend on its parent
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ assertEquals(VertexState.NEW, vertex3.getState());
+ // no VertexEvent pass to downstream vertex
+ assertEquals(0, vertexEventHandler.getEvents().size());
+ }
+
+ private TezEvent createTezEvent() {
+ return new TezEvent(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.allocate(0)),
+ new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", null,
+ null));
+ }
+
+ /**
+ * vertex1(New) -> restoreFromDataMovementEvent -> StartRecoveryTransition
+ */
+ @Test
+ public void testRecovery_New_Desired_RUNNING() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ VertexState recoveredState =
+ vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
+ assertEquals(VertexState.NEW, recoveredState);
+ assertEquals(1, vertex1.recoveredEvents.size());
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+
+ // InputDataInformationEvent is removed
+ assertEquals(0, vertex1.recoveredEvents.size());
+ // V_INIT and V_START is sent
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+
+ }
+
+ private void assertTaskRecoveredEventSent(VertexImpl vertex) {
+ int sentNum = 0;
+ for (TaskEvent event : taskEventHandler.getEvents()) {
+ if (event.getType() == TaskEventType.T_RECOVER) {
+ TaskEventRecoverTask recoverEvent = (TaskEventRecoverTask)event;
+ if (recoverEvent.getTaskID().getVertexID().equals(vertex.getVertexId())){
+ sentNum++;
+ }
+ }
+ }
+ assertEquals("expect " + vertex.getTotalTasks()
+ + " TaskEventTaskRecover sent for vertex:" + vertex.getVertexId() +
+ "but actuall sent " + sentNum, vertex.getTotalTasks(), sentNum);
+ }
+
+ private void assertOutputCommitters(VertexImpl vertex){
+ assertTrue(vertex.getOutputCommitters() != null);
+ for (OutputCommitter c : vertex.getOutputCommitters().values()) {
+ CountingOutputCommitter committer = (CountingOutputCommitter) c;
+ assertEquals(0, committer.abortCounter);
+ assertEquals(0, committer.commitCounter);
+ assertEquals(1, committer.initCounter);
+ assertEquals(1, committer.setupCounter);
+ }
+ }
+
+ private void restoreFromInitializedEvent(VertexImpl vertex) {
+ long initTimeRequested = 100L;
+ long initedTime = initTimeRequested + 100L;
+ VertexState recoveredState =
+ vertex.restoreFromEvent(new VertexInitializedEvent(vertex
+ .getVertexId(), "vertex1", initTimeRequested, initedTime, vertex.getTotalTasks(),
+ "", null));
+ assertEquals(VertexState.INITED, recoveredState);
+ assertEquals(vertex.getTotalTasks(), vertex.getTasks().size());
+ assertEquals(initTimeRequested, vertex.initTimeRequested);
+ assertEquals(initedTime, vertex.initedTime);
+ }
+
+ /**
+ * restoreFromVertexInitializedEvent -> StartRecoveryTransition
+ */
+ @Test
+ public void testRecovery_Inited_Desired_RUNNING() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ restoreFromInitializedEvent(vertex1);
+
+ VertexState recoveredState =
+ vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
+ assertEquals(VertexState.INITED, recoveredState);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+
+ // InputDataInformationEvent is removed
+ assertEquals(0, vertex1.recoveredEvents.size());
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+ // task recovered event is sent
+ assertTaskRecoveredEventSent(vertex1);
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+ }
+
+ /**
+ * restoreFromVertexInitializedEvent -> restoreFromVertexStartedEvent ->
+ * StartRecoveryTransition
+ */
+ @Test
+ public void testRecovery_Started_Desired_RUNNING() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ restoreFromInitializedEvent(vertex1);
+
+ long startTimeRequested = initedTime + 100L;
+ long startedTime = startTimeRequested + 100L;
+ VertexState recoveredState =
+ vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+ startTimeRequested, startedTime));
+ assertEquals(VertexState.RUNNING, recoveredState);
+ assertEquals(startTimeRequested, vertex1.startTimeRequested);
+ assertEquals(startedTime, vertex1.startedTime);
+
+ recoveredState =
+ vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
+ assertEquals(VertexState.RUNNING, recoveredState);
+ assertEquals(1, vertex1.recoveredEvents.size());
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+
+ // InputDataInformationEvent is removed
+ assertEquals(0, vertex1.recoveredEvents.size());
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+ // task recovered event is sent
+ assertTaskRecoveredEventSent(vertex1);
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+ }
+
+ /**
+ * restoreFromVertexInitializedEvent -> restoreFromVertexStartedEvent ->
+ * restoreFromVertexFinishedEvent -> StartRecoveryTransition
+ */
+ @Test
+ public void testRecovery_Finished_Desired_RUNNING() {
+ // v1: initFromInitializedEvent
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ restoreFromInitializedEvent(vertex1);
+
+ // v1: initFromStartedEvent
+ long startRequestedTime = initedTime + 100L;
+ long startTime = startRequestedTime + 100L;
+ VertexState recoveredState =
+ vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+ startRequestedTime, startTime));
+ assertEquals(VertexState.RUNNING, recoveredState);
+
+ // v1: initFromFinishedEvent
+ long finishTime = startTime + 100L;
+ recoveredState =
+ vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
+ "vertex1", initRequestedTime, initedTime, startRequestedTime,
+ startTime, finishTime, VertexState.SUCCEEDED, "",
+ new TezCounters(), new VertexStats()));
+ assertEquals(finishTime, vertex1.finishTime);
+ assertEquals(VertexState.SUCCEEDED, recoveredState);
+ assertEquals(false, vertex1.recoveryCommitInProgress);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+
+ // InputDataInformationEvent is removed
+ assertEquals(0, vertex1.recoveredEvents.size());
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+ // task recovered event is sent
+ assertTaskRecoveredEventSent(vertex1);
+
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+ }
+
+ /**
+ * vertex1 (New) -> StartRecoveryTransition <br>
+ * vertex2 (New) -> StartRecoveryTransition vertex3 (New) -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_RecoveringFromNew() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+ assertEquals(1, vertex1.getTasks().size());
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ VertexState recoveredState =
+ vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+ assertEquals(VertexState.NEW, recoveredState);
+ assertEquals(1, vertex3.recoveredEvents.size());
+
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+
+ VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+ vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex2.getState());
+ // no OutputCommitter for vertex2
+ assertNull(vertex2.getOutputCommitters());
+
+ // v3 go to RUNNING because v1 and v2 both start
+ assertEquals(VertexState.RUNNING, vertex3.getState());
+ assertEquals(2, vertex3.numRecoveredSourceVertices);
+ assertEquals(2, vertex3.numInitedSourceVertices);
+ assertEquals(2, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+ // RootInputDataInformation is removed
+ assertEquals(0, vertex3.recoveredEvents.size());
+
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex3);
+
+ }
+
+ /**
+ * vertex1 (New) -> restoreFromInitialized -> StartRecoveryTransition<br>
+ * vertex2 (New) -> restoreFromInitialized -> StartRecoveryTransition<br>
+ * vertex3 (New) -> restoreFromVertexInitedEvent -> RecoverTransition<br>
+ */
+ @Test
+ public void testRecovery_RecoveringFromInited() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ restoreFromInitializedEvent(vertex1);
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+ assertEquals(vertex1.getTotalTasks(), vertex1.getTasks().size());
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ VertexState recoveredState =
+ vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+ assertEquals(VertexState.NEW, recoveredState);
+ assertEquals(1, vertex3.recoveredEvents.size());
+ recoveredState =
+ vertex3.restoreFromEvent(new VertexInitializedEvent(vertex3
+ .getVertexId(), "vertex3", initRequestedTime, initedTime, 2, "",
+ null));
+ assertEquals(VertexState.INITED, recoveredState);
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+
+ VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+ restoreFromInitializedEvent(vertex2);
+ vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex2.getState());
+
+ // v3 go to RUNNING because v1 and v2 both start
+ assertEquals(VertexState.RUNNING, vertex3.getState());
+ assertEquals(2, vertex3.numRecoveredSourceVertices);
+ // numInitedSourceVertices is wrong but doesn't matter because v3 has
+ // already initialized
+ assertEquals(2, vertex3.numInitedSourceVertices);
+ assertEquals(2, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+ // RootInputDataInformation is removed
+ assertEquals(0, vertex3.recoveredEvents.size());
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex3);
+ // 1 for vertex1, 2 for vertex2, the second 2 for vertex3
+ assertTaskRecoveredEventSent(vertex1);
+ assertTaskRecoveredEventSent(vertex2);
+ assertTaskRecoveredEventSent(vertex3);
+ }
+
+ /**
+ * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
+ * StartRecoveryTransition <br>
+ * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> StartRecoveryTransition <br>
+ * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_RecoveringFromRunning() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ restoreFromInitializedEvent(vertex1);
+ VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+ initRequestedTime + 100L, initRequestedTime + 200L));
+ assertEquals(VertexState.RUNNING, recoveredState);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+ assertEquals(1, vertex1.getTasks().size());
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ recoveredState =
+ vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+ assertEquals(VertexState.NEW, recoveredState);
+ assertEquals(1, vertex3.recoveredEvents.size());
+ recoveredState =
+ vertex3.restoreFromEvent(new VertexInitializedEvent(vertex3
+ .getVertexId(), "vertex3", initRequestedTime, initedTime, vertex3.getTotalTasks(), "",
+ null));
+ assertEquals(VertexState.INITED, recoveredState);
+ recoveredState =
+ vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
+ initRequestedTime + 100L, initRequestedTime + 200L));
+ assertEquals(VertexState.RUNNING, recoveredState);
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+
+ VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+ recoveredState = vertex2.restoreFromEvent(new VertexInitializedEvent(vertex2.getVertexId(),
+ "vertex2", initRequestedTime, initedTime, vertex2.getTotalTasks(), "", null));
+ assertEquals(VertexState.INITED, recoveredState);
+ recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
+ initRequestedTime + 100L, initRequestedTime + 200L));
+ assertEquals(VertexState.RUNNING, recoveredState);
+
+ vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex2.getState());
+
+ // v3 go to RUNNING because v1 and v2 both start
+ assertEquals(VertexState.RUNNING, vertex3.getState());
+ assertEquals(2, vertex3.numRecoveredSourceVertices);
+ assertEquals(2, vertex3.numInitedSourceVertices);
+ assertEquals(2, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+ // RootInputDataInformation is removed
+ assertEquals(0, vertex3.recoveredEvents.size());
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex3);
+
+ assertTaskRecoveredEventSent(vertex1);
+ assertTaskRecoveredEventSent(vertex2);
+ assertTaskRecoveredEventSent(vertex3);
+ }
+
+ /**
+ * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
+ * restoreFromVertexFinished -> StartRecoveryTransition<br>
+ * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
+ * restoreFromVertexFinished -> StartRecoveryTransition<br>
+ * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> RecoverTransition
+ */
+ @Test
+ public void testRecovery_RecoveringFromSUCCEEDED() {
+ VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+ restoreFromInitializedEvent(vertex1);
+ VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+ initRequestedTime + 100L, initRequestedTime + 200L));
+ assertEquals(VertexState.RUNNING, recoveredState);
+
+ recoveredState = vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
+ "vertex1", initRequestedTime, initedTime, initRequestedTime + 300L,
+ initRequestedTime + 400L, initRequestedTime + 500L,
+ VertexState.SUCCEEDED, "", new TezCounters(), new VertexStats()));
+ assertEquals(VertexState.SUCCEEDED, recoveredState);
+
+ vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex1.getState());
+ assertEquals(1, vertex1.getTasks().size());
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex1);
+
+ VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+ recoveredState =
+ vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+ assertEquals(VertexState.NEW, recoveredState);
+ assertEquals(1, vertex3.recoveredEvents.size());
+ restoreFromInitializedEvent(vertex3);
+ recoveredState =
+ vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
+ initRequestedTime + 100L, initRequestedTime + 200L));
+ assertEquals(VertexState.RUNNING, recoveredState);
+ // wait for recovery of vertex2
+ assertEquals(VertexState.RECOVERING, vertex3.getState());
+ assertEquals(1, vertex3.numRecoveredSourceVertices);
+ assertEquals(1, vertex3.numInitedSourceVertices);
+ assertEquals(1, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+
+ VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+ recoveredState = vertex2.restoreFromEvent(new VertexInitializedEvent(vertex2.getVertexId(),
+ "vertex2", initRequestedTime, initedTime, vertex2.getTotalTasks(), "", null));
+ assertEquals(VertexState.INITED, recoveredState);
+ recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
+ initRequestedTime + 100L, initRequestedTime + 200L));
+ assertEquals(VertexState.RUNNING, recoveredState);
+ vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+ VertexState.RUNNING));
+ dispatcher.await();
+ assertEquals(VertexState.RUNNING, vertex2.getState());
+
+ // v3 go to RUNNING because v1 and v2 both start
+ assertEquals(VertexState.RUNNING, vertex3.getState());
+ assertEquals(2, vertex3.numRecoveredSourceVertices);
+ assertEquals(2, vertex3.numInitedSourceVertices);
+ assertEquals(2, vertex3.numStartedSourceVertices);
+ assertEquals(1, vertex3.getDistanceFromRoot());
+ // RootInputDataInformation is removed
+ assertEquals(0, vertex3.recoveredEvents.size());
+ // verify OutputCommitter is initialized
+ assertOutputCommitters(vertex3);
+
+ assertTaskRecoveredEventSent(vertex1);
+ assertTaskRecoveredEventSent(vertex2);
+ assertTaskRecoveredEventSent(vertex3);
+ }
+}
[16/25] git commit: TEZ-1580. Change TestOrderedWordCount to
optionally use MR configs. (hitesh)
Posted by je...@apache.org.
TEZ-1580. Change TestOrderedWordCount to optionally use MR configs. (hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/9dd0cb4d
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/9dd0cb4d
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/9dd0cb4d
Branch: refs/heads/TEZ-8
Commit: 9dd0cb4d8e933d5f57b3d9ae532e7167978aed68
Parents: 5e5683a
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 14:25:17 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 14:25:17 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../examples/TestOrderedWordCount.java | 61 +++++++++++++++-----
2 files changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/9dd0cb4d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 73a3671..f71c2e2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -16,6 +16,7 @@ ALL CHANGES:
TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts.
TEZ-1578. Remove TeraSort from Tez codebase.
TEZ-1569. Add tests for preemption
+ TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/9dd0cb4d/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
index 2c5db10..a36d1d2 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
@@ -149,7 +149,9 @@ public class TestOrderedWordCount extends Configured implements Tool {
public DAG createDAG(FileSystem fs, Configuration conf,
Map<String, LocalResource> commonLocalResources, Path stagingDir,
int dagIndex, String inputPath, String outputPath,
- boolean generateSplitsInClient) throws Exception {
+ boolean generateSplitsInClient,
+ boolean useMRSettings,
+ int intermediateNumReduceTasks) throws Exception {
Configuration mapStageConf = new JobConf(conf);
mapStageConf.set(MRJobConfig.MAP_CLASS_ATTR,
@@ -196,32 +198,62 @@ public class TestOrderedWordCount extends Configured implements Tool {
dsd = MRInputLegacy.createConfigBuilder(mapStageConf, TextInputFormat.class, inputPath).build();
}
- Vertex mapVertex = Vertex.create("initialmap", ProcessorDescriptor.create(
- MapProcessor.class.getName()).setUserPayload(
- TezUtils.createUserPayloadFromConf(mapStageConf))
- .setHistoryText(mapStageHistoryText)).addTaskLocalFiles(commonLocalResources);
- mapVertex.addDataSource("MRInput", dsd);
+ Vertex mapVertex;
+ ProcessorDescriptor mapProcessorDescriptor =
+ ProcessorDescriptor.create(MapProcessor.class.getName())
+ .setUserPayload(
+ TezUtils.createUserPayloadFromConf(mapStageConf))
+ .setHistoryText(mapStageHistoryText);
+ if (!useMRSettings) {
+ mapVertex = Vertex.create("initialmap", mapProcessorDescriptor);
+ } else {
+ mapVertex = Vertex.create("initialmap", mapProcessorDescriptor, -1,
+ MRHelpers.getResourceForMRMapper(mapStageConf));
+ mapVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
+ }
+ mapVertex.addTaskLocalFiles(commonLocalResources)
+ .addDataSource("MRInput", dsd);
vertices.add(mapVertex);
ByteArrayOutputStream iROutputStream = new ByteArrayOutputStream(4096);
iReduceStageConf.writeXml(iROutputStream);
String iReduceStageHistoryText = new String(iROutputStream.toByteArray(), "UTF-8");
- Vertex ivertex = Vertex.create("intermediate_reducer", ProcessorDescriptor.create(
+
+ ProcessorDescriptor iReduceProcessorDescriptor = ProcessorDescriptor.create(
ReduceProcessor.class.getName())
.setUserPayload(TezUtils.createUserPayloadFromConf(iReduceStageConf))
- .setHistoryText(iReduceStageHistoryText), 2);
- ivertex.addTaskLocalFiles(commonLocalResources);
- vertices.add(ivertex);
+ .setHistoryText(iReduceStageHistoryText);
+
+ Vertex intermediateVertex;
+ if (!useMRSettings) {
+ intermediateVertex = Vertex.create("intermediate_reducer", iReduceProcessorDescriptor,
+ intermediateNumReduceTasks);
+ } else {
+ intermediateVertex = Vertex.create("intermediate_reducer", iReduceProcessorDescriptor,
+ intermediateNumReduceTasks, MRHelpers.getResourceForMRReducer(iReduceStageConf));
+ intermediateVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(iReduceStageConf));
+ }
+ intermediateVertex.addTaskLocalFiles(commonLocalResources);
+ vertices.add(intermediateVertex);
ByteArrayOutputStream finalReduceOutputStream = new ByteArrayOutputStream(4096);
finalReduceConf.writeXml(finalReduceOutputStream);
String finalReduceStageHistoryText = new String(finalReduceOutputStream.toByteArray(), "UTF-8");
UserPayload finalReducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
- Vertex finalReduceVertex = Vertex.create("finalreduce",
+ Vertex finalReduceVertex;
+
+ ProcessorDescriptor finalReduceProcessorDescriptor =
ProcessorDescriptor.create(
ReduceProcessor.class.getName())
.setUserPayload(finalReducePayload)
- .setHistoryText(finalReduceStageHistoryText), 1);
+ .setHistoryText(finalReduceStageHistoryText);
+ if (!useMRSettings) {
+ finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1);
+ } else {
+ finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1,
+ MRHelpers.getResourceForMRReducer(finalReduceConf));
+ finalReduceVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
+ }
finalReduceVertex.addTaskLocalFiles(commonLocalResources);
finalReduceVertex.addDataSink("MROutput",
MROutputLegacy.createConfigBuilder(finalReduceConf, TextOutputFormat.class, outputPath)
@@ -283,6 +315,9 @@ public class TestOrderedWordCount extends Configured implements Tool {
* 1000;
boolean retainStagingDir = conf.getBoolean("RETAIN_STAGING_DIR", false);
+ boolean useMRSettings = conf.getBoolean("USE_MR_CONFIGS", true);
+ // TODO needs to use auto reduce parallelism
+ int intermediateNumReduceTasks = conf.getInt("IREDUCE_NUM_TASKS", 2);
if (((otherArgs.length%2) != 0)
|| (!useTezSession && otherArgs.length != 2)) {
@@ -371,7 +406,7 @@ public class TestOrderedWordCount extends Configured implements Tool {
DAG dag = instance.createDAG(fs, conf, localResources,
stagingDir, dagIndex, inputPath, outputPath,
- generateSplitsInClient);
+ generateSplitsInClient, useMRSettings, intermediateNumReduceTasks);
boolean doPreWarm = dagIndex == 1 && useTezSession
&& conf.getBoolean("PRE_WARM_SESSION", true);
[03/25] git commit: Revert "TEZ-1494. DAG hangs waiting for
ShuffleManager.getNextInput() (Rajesh Balamohan)"
Posted by je...@apache.org.
Revert "TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput() (Rajesh Balamohan)"
This reverts commit de008b54a208881321d15addfe35b8094720be25.
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/80b5795b
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/80b5795b
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/80b5795b
Branch: refs/heads/TEZ-8
Commit: 80b5795b13b7483d876b741245708a56794cd771
Parents: f65e65a
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Thu Sep 11 03:07:16 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Sep 12 03:54:46 2014 +0530
----------------------------------------------------------------------
.../dag/impl/ImmediateStartVertexManager.java | 92 +----
.../app/dag/impl/RootInputVertexManager.java | 27 +-
.../tez/dag/app/dag/impl/TestVertexImpl.java | 364 ++-----------------
3 files changed, 66 insertions(+), 417 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/80b5795b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
index ac2b851..b202d70 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
@@ -18,11 +18,9 @@
package org.apache.tez.dag.app.dag.impl;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.dag.api.EdgeProperty;
+import java.util.List;
+import java.util.Map;
+
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.VertexManagerPlugin;
import org.apache.tez.dag.api.VertexManagerPluginContext;
@@ -30,101 +28,29 @@ import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
import org.apache.tez.runtime.api.Event;
import org.apache.tez.runtime.api.events.VertexManagerEvent;
-import java.util.List;
-import java.util.Map;
+import com.google.common.collect.Lists;
/**
* Starts all tasks immediately on vertex start
*/
public class ImmediateStartVertexManager extends VertexManagerPlugin {
- private static final Log LOG = LogFactory.getLog(ImmediateStartVertexManager.class);
-
- private final Map<String, SourceVertexInfo> srcVertexInfo = Maps.newHashMap();
- private int managedTasks;
- private boolean tasksScheduled = false;
-
- class SourceVertexInfo {
- EdgeProperty edgeProperty;
- int numFinishedTasks;
-
- SourceVertexInfo(EdgeProperty edgeProperty) {
- this.edgeProperty = edgeProperty;
- }
- }
-
public ImmediateStartVertexManager(VertexManagerPluginContext context) {
super(context);
}
@Override
public void onVertexStarted(Map<String, List<Integer>> completions) {
- managedTasks = getContext().getVertexNumTasks(getContext().getVertexName());
- Map<String, EdgeProperty> edges = getContext().getInputVertexEdgeProperties();
- for (Map.Entry<String, EdgeProperty> entry : edges.entrySet()) {
- String srcVertex = entry.getKey();
- EdgeProperty edgeProp = entry.getValue();
- srcVertexInfo.put(srcVertex, new SourceVertexInfo(edgeProp));
- }
-
- //handle completions
- for (Map.Entry<String, List<Integer>> entry : completions.entrySet()) {
- for (Integer task : entry.getValue()) {
- handleSourceTaskFinished(entry.getKey(), task);
- }
- }
- scheduleTasks();
- }
-
- private void handleSourceTaskFinished(String vertex, Integer taskId) {
- SourceVertexInfo srcInfo = srcVertexInfo.get(vertex);
- //Not mandatory to check for duplicate completions here
- srcInfo.numFinishedTasks++;
- }
-
- private void scheduleTasks() {
- if (!canScheduleTasks()) {
- return;
- }
-
- List<TaskWithLocationHint> tasksToStart = Lists.newArrayListWithCapacity(managedTasks);
- for (int i = 0; i < managedTasks; ++i) {
- tasksToStart.add(new TaskWithLocationHint(new Integer(i), null));
- }
-
- if (!tasksToStart.isEmpty()) {
- LOG.info("Starting " + tasksToStart.size() + " in " + getContext().getVertexName());
- getContext().scheduleVertexTasks(tasksToStart);
+ int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
+ List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
+ for (int i=0; i<numTasks; ++i) {
+ scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
}
- tasksScheduled = true;
- }
-
- private boolean canScheduleTasks() {
- //Check if at least 1 task is finished from each source vertex (in case of broadcast &
- // one-to-one or custom)
- for (Map.Entry<String, SourceVertexInfo> entry : srcVertexInfo.entrySet()) {
- SourceVertexInfo srcVertexInfo = entry.getValue();
- switch(srcVertexInfo.edgeProperty.getDataMovementType()) {
- case ONE_TO_ONE:
- case BROADCAST:
- case CUSTOM:
- if (srcVertexInfo.numFinishedTasks == 0) {
- //do not schedule tasks until a task from source task is complete
- return false;
- }
- default:
- break;
- }
- }
- return true;
+ getContext().scheduleVertexTasks(scheduledTasks);
}
@Override
public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
- handleSourceTaskFinished(srcVertexName, attemptId);
- if (!tasksScheduled) {
- scheduleTasks();
- }
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/80b5795b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
index e850286..e6ffdc5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
@@ -18,23 +18,27 @@
package org.apache.tez.dag.app.dag.impl;
+import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.VertexManagerPlugin;
import org.apache.tez.dag.api.VertexManagerPluginContext;
+import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
import org.apache.tez.runtime.api.Event;
import org.apache.tez.runtime.api.InputSpecUpdate;
import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
import org.apache.tez.runtime.api.events.InputDataInformationEvent;
import org.apache.tez.runtime.api.events.InputUpdatePayloadEvent;
+import org.apache.tez.runtime.api.events.VertexManagerEvent;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
-public class RootInputVertexManager extends ImmediateStartVertexManager {
+public class RootInputVertexManager extends VertexManagerPlugin {
private String configuredInputName;
@@ -42,6 +46,27 @@ public class RootInputVertexManager extends ImmediateStartVertexManager {
super(context);
}
+ @Override
+ public void initialize() {
+ }
+
+ @Override
+ public void onVertexStarted(Map<String, List<Integer>> completions) {
+ int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
+ List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
+ for (int i=0; i<numTasks; ++i) {
+ scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
+ }
+ getContext().scheduleVertexTasks(scheduledTasks);
+ }
+
+ @Override
+ public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
+ }
+
+ @Override
+ public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) {
+ }
@Override
public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor,
http://git-wip-us.apache.org/repos/asf/tez/blob/80b5795b/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 04e2219..d894928 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
@@ -20,7 +20,6 @@ package org.apache.tez.dag.app.dag.impl;
import java.nio.ByteBuffer;
-import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Mockito.doReturn;
@@ -1880,39 +1879,39 @@ public class TestVertexImpl {
Assert.assertEquals(2, v3.getOutputVerticesCount());
Assert.assertEquals(2, v3.getOutputVerticesCount());
- assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
+ Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
.getSourceVertexName())
|| "vertex2".equals(v3.getInputSpecList(0).get(0)
- .getSourceVertexName()));
- assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
+ .getSourceVertexName()));
+ Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
.getSourceVertexName())
|| "vertex2".equals(v3.getInputSpecList(0).get(1)
- .getSourceVertexName()));
- assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
+ .getSourceVertexName()));
+ Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
.getInputDescriptor().getClassName())
|| "i3_v2".equals(v3.getInputSpecList(0).get(0)
- .getInputDescriptor().getClassName()));
- assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
+ .getInputDescriptor().getClassName()));
+ Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
.getInputDescriptor().getClassName())
|| "i3_v2".equals(v3.getInputSpecList(0).get(1)
- .getInputDescriptor().getClassName()));
+ .getInputDescriptor().getClassName()));
- assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
+ Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
.getDestinationVertexName())
|| "vertex5".equals(v3.getOutputSpecList(0).get(0)
- .getDestinationVertexName()));
- assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
+ .getDestinationVertexName()));
+ Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
.getDestinationVertexName())
|| "vertex5".equals(v3.getOutputSpecList(0).get(1)
- .getDestinationVertexName()));
- assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
+ .getDestinationVertexName()));
+ Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
.getOutputDescriptor().getClassName())
|| "o3_v5".equals(v3.getOutputSpecList(0).get(0)
- .getOutputDescriptor().getClassName()));
- assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
+ .getOutputDescriptor().getClassName()));
+ Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
.getOutputDescriptor().getClassName())
|| "o3_v5".equals(v3.getOutputSpecList(0).get(1)
- .getOutputDescriptor().getClassName()));
+ .getOutputDescriptor().getClassName()));
}
@Test(timeout = 5000)
@@ -1941,13 +1940,13 @@ public class TestVertexImpl {
Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
Collections.singletonMap(
v1.getName(), mockEdgeManagerDescriptor);
- assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
- assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
+ Assert.assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
+ Assert.assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
EdgeManagerForTest);
Assert.assertEquals(1, v3.getTotalTasks());
Assert.assertEquals(1, tasks.size());
// the last one is removed
- assertTrue(tasks.keySet().iterator().next().equals(firstTask));
+ Assert.assertTrue(tasks.keySet().iterator().next().equals(firstTask));
}
@@ -1994,7 +1993,7 @@ public class TestVertexImpl {
Edge edge = edges.get("e4");
EdgeManagerPlugin em = edge.getEdgeManager();
EdgeManagerForTest originalEm = (EdgeManagerForTest) em;
- assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
+ Assert.assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
.getUserPayload().deepCopyAsArray()));
UserPayload userPayload = UserPayload.create(ByteBuffer.wrap(new String("foo").getBytes()));
@@ -2008,7 +2007,7 @@ public class TestVertexImpl {
Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
Collections.singletonMap(v3.getName(), edgeManagerDescriptor);
- assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
+ Assert.assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
edgeManagerDescriptors, null)); // Must decrease.
VertexImpl v5Impl = (VertexImpl) v5;
@@ -2016,10 +2015,10 @@ public class TestVertexImpl {
EdgeManagerPlugin modifiedEdgeManager = v5Impl.sourceVertices.get(v3)
.getEdgeManager();
Assert.assertNotNull(modifiedEdgeManager);
- assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
+ Assert.assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
// Ensure initialize() is called with the correct payload
- assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
+ Assert.assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
((EdgeManagerForTest) modifiedEdgeManager).getUserPayload().deepCopyAsArray()));
}
@@ -2093,7 +2092,7 @@ public class TestVertexImpl {
Assert.assertEquals(VertexTerminationCause.OWN_TASK_FAILURE, v.getTerminationCause());
String diagnostics =
StringUtils.join(v.getDiagnostics(), ",").toLowerCase();
- assertTrue(diagnostics.contains("task failed"
+ Assert.assertTrue(diagnostics.contains("task failed"
+ ", taskid=" + t1.toString()));
}
@@ -2105,7 +2104,7 @@ public class TestVertexImpl {
String diagnostics =
StringUtils.join(v2.getDiagnostics(), ",").toLowerCase();
LOG.info("diagnostics v2: " + diagnostics);
- assertTrue(diagnostics.contains(
+ Assert.assertTrue(diagnostics.contains(
"vertex received kill in inited state"));
}
@@ -2119,7 +2118,7 @@ public class TestVertexImpl {
killVertex(v3);
String diagnostics =
StringUtils.join(v3.getDiagnostics(), ",").toLowerCase();
- assertTrue(diagnostics.contains(
+ Assert.assertTrue(diagnostics.contains(
"vertex received kill while in running state"));
}
@@ -2201,7 +2200,7 @@ public class TestVertexImpl {
Assert.assertNull(v2.getOutputCommitter("output"));
VertexImpl v6 = vertices.get("vertex6");
- assertTrue(v6.getOutputCommitter("outputx")
+ Assert.assertTrue(v6.getOutputCommitter("outputx")
instanceof CountingOutputCommitter);
}
@@ -2209,11 +2208,11 @@ public class TestVertexImpl {
public void testVertexManagerInit() {
initAllVertices(VertexState.INITED);
VertexImpl v2 = vertices.get("vertex2");
- assertTrue(v2.getVertexManager().getPlugin()
+ Assert.assertTrue(v2.getVertexManager().getPlugin()
instanceof ImmediateStartVertexManager);
VertexImpl v6 = vertices.get("vertex6");
- assertTrue(v6.getVertexManager().getPlugin()
+ Assert.assertTrue(v6.getVertexManager().getPlugin()
instanceof ShuffleVertexManager);
}
@@ -3021,306 +3020,6 @@ public class TestVertexImpl {
}
}
- @Test(timeout = 5000)
- /**
- * Ref: TEZ-1494
- * If broadcast, one-to-one or custom edges are present in source, tasks should not start until
- * 1 task from each source vertex is complete.
- */
- public void testTaskSchedulingWithCustomEdges() {
- setupPreDagCreation();
- dagPlan = createCustomDAGWithCustomEdges();
- setupPostDagCreation();
-
- /**
- *
- * M2 --(SG)--> R3 --(B)--\
- * \
- * M7 --(B)---------------->M5 ---(SG)--> R6
- * /
- * M8---(C)--------------->/
- */
-
- //init M2
- VertexImpl m2 = vertices.get("M2");
- VertexImpl m7 = vertices.get("M7");
- VertexImpl r3 = vertices.get("R3");
- VertexImpl m5 = vertices.get("M5");
- VertexImpl m8 = vertices.get("M8");
-
- initVertex(m2);
- initVertex(m7);
- initVertex(m8);
- assertTrue(m7.getState().equals(VertexState.INITED));
- assertTrue(m5.getState().equals(VertexState.INITED));
- assertTrue(m8.getState().equals(VertexState.INITED));
- assertTrue(m7.getVertexManager().getPlugin() instanceof ImmediateStartVertexManager);
-
- //Start M2; Let tasks complete in M2; Also let 1 task complete in R3
- dispatcher.getEventHandler().handle(new VertexEvent(m2.getVertexId(), VertexEventType.V_START));
- dispatcher.await();
- VertexEventTaskAttemptCompleted taskAttemptCompleted = new VertexEventTaskAttemptCompleted
- (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m2.getVertexId(),0), 0), TaskAttemptStateInternal.SUCCEEDED);
- VertexEventTaskCompleted taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m2
- .getVertexId(), 0), TaskState.SUCCEEDED);
- dispatcher.getEventHandler().handle(taskAttemptCompleted);
- dispatcher.getEventHandler().handle(taskCompleted);
- dispatcher.await();
- taskAttemptCompleted = new VertexEventTaskAttemptCompleted
- (TezTaskAttemptID.getInstance(TezTaskID.getInstance(r3.getVertexId(),0), 0),
- TaskAttemptStateInternal.SUCCEEDED);
- taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(r3
- .getVertexId(), 0), TaskState.SUCCEEDED);
- dispatcher.getEventHandler().handle(taskAttemptCompleted);
- dispatcher.getEventHandler().handle(taskCompleted);
- dispatcher.await();
- assertTrue(m2.getState().equals(VertexState.SUCCEEDED));
- assertTrue(m5.numSuccessSourceAttemptCompletions == 1);
- assertTrue(m5.getState().equals(VertexState.INITED));
-
- //R3 should be in running state as it has one task completed, and rest are pending
- assertTrue(r3.getState().equals(VertexState.RUNNING));
-
- //Let us start M7; M5 should start not start as it is dependent on M8 as well
- dispatcher.getEventHandler().handle(new VertexEvent(m7.getVertexId(),VertexEventType.V_START));
- dispatcher.await();
- //Let one of the tasks get over in M7 as well.
- taskAttemptCompleted = new VertexEventTaskAttemptCompleted
- (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m7.getVertexId(),0), 0),
- TaskAttemptStateInternal.SUCCEEDED);
- taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m7
- .getVertexId(), 0), TaskState.SUCCEEDED);
- dispatcher.getEventHandler().handle(taskAttemptCompleted);
- dispatcher.getEventHandler().handle(taskCompleted);
- dispatcher.await();
- assertTrue(m5.numSuccessSourceAttemptCompletions == 2);
-
- //M5 should be in INITED state, as it depends on M8
- assertTrue(m5.getState().equals(VertexState.INITED));
- for(Task task : m5.getTasks().values()) {
- assertTrue(task.getState().equals(TaskState.NEW));
- }
-
- //Let us start M8; M5 should start now
- dispatcher.getEventHandler().handle(new VertexEvent(m8.getVertexId(),VertexEventType.V_START));
- dispatcher.await();
-
- //M5 in running state. But tasks should not be scheduled until M8 finishes a task.
- assertTrue(m5.getState().equals(VertexState.RUNNING));
- for(Task task : m5.getTasks().values()) {
- assertTrue(task.getState().equals(TaskState.NEW));
- }
-
- //Let one of the tasks get over in M8 as well. This should trigger tasks to be scheduled in M5
- taskAttemptCompleted = new VertexEventTaskAttemptCompleted
- (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m8.getVertexId(),0), 0),
- TaskAttemptStateInternal.SUCCEEDED);
- taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m8
- .getVertexId(), 0), TaskState.SUCCEEDED);
- dispatcher.getEventHandler().handle(taskAttemptCompleted);
- dispatcher.getEventHandler().handle(taskCompleted);
- dispatcher.await();
-
- assertTrue(m5.numSuccessSourceAttemptCompletions == 3);
- //Ensure all tasks in M5 are in scheduled state
- for(Task task : m5.getTasks().values()) {
- assertTrue(task.getState().equals(TaskState.SCHEDULED));
- }
- }
-
- //For TEZ-1494
- private DAGPlan createCustomDAGWithCustomEdges() {
- /**
- *
- * M2 --(SG)--> R3 --(B)--\
- * \
- * M7 --(B)---------------->M5 ---(SG)--> R6
- * /
- * M8---(C)--------------->/
- */
- DAGPlan dag = DAGPlan.newBuilder().setName("TestSamplerDAG")
- .addVertex(VertexPlan.newBuilder()
- .setName("M2")
- .setProcessorDescriptor(
- TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
- .setType(PlanVertexType.NORMAL)
- .addTaskLocationHint(
- PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
- .setTaskConfig(PlanTaskConfiguration.newBuilder()
- .setNumTasks(1)
- .setVirtualCores(4)
- .setMemoryMb(1024)
- .setJavaOpts("")
- .setTaskModule("M2.class")
- .build()
- )
- .addOutEdgeId("M2_R3")
- .build()
- )
- .addVertex(VertexPlan.newBuilder()
- .setName("M8")
- .setProcessorDescriptor(
- TezEntityDescriptorProto.newBuilder().setClassName("M8.class"))
- .setType(PlanVertexType.NORMAL)
- .addTaskLocationHint(
- PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
- .setTaskConfig(PlanTaskConfiguration.newBuilder()
- .setNumTasks(1)
- .setVirtualCores(4)
- .setMemoryMb(1024)
- .setJavaOpts("")
- .setTaskModule("M8.class")
- .build()
- )
- .addOutEdgeId("M8_M5")
- .build()
- )
- .addVertex(VertexPlan.newBuilder()
- .setName("R3")
- .setProcessorDescriptor(
- TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
- .setType(PlanVertexType.NORMAL)
- .addTaskLocationHint(
- PlanTaskLocationHint.newBuilder().addHost("host2").addRack("rack1").build())
- .setTaskConfig(PlanTaskConfiguration.newBuilder()
- .setNumTasks(10)
- .setVirtualCores(4)
- .setMemoryMb(1024)
- .setJavaOpts("")
- .setTaskModule("R3.class")
- .build()
- )
- .addInEdgeId("M2_R3")
- .addOutEdgeId("R3_M5")
- .build()
- )
- .addVertex(VertexPlan.newBuilder()
- .setName("M5")
- .setProcessorDescriptor(
- TezEntityDescriptorProto.newBuilder().setClassName("M5.class"))
- .setType(PlanVertexType.NORMAL)
- .addTaskLocationHint(
- PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
- .setTaskConfig(PlanTaskConfiguration.newBuilder()
- .setNumTasks(10)
- .setVirtualCores(4)
- .setMemoryMb(1024)
- .setJavaOpts("")
- .setTaskModule("M5.class")
- .build()
- )
- .addInEdgeId("R3_M5")
- .addInEdgeId("M7_M5")
- .addInEdgeId("M8_M5")
- .addOutEdgeId("M5_R6")
- .build()
- )
- .addVertex(VertexPlan.newBuilder()
- .setName("M7")
- .setProcessorDescriptor(
- TezEntityDescriptorProto.newBuilder().setClassName("M7.class"))
- .setType(PlanVertexType.NORMAL)
- .addTaskLocationHint(
- PlanTaskLocationHint.newBuilder().addHost("host4").addRack("rack1").build())
- .setTaskConfig(PlanTaskConfiguration.newBuilder()
- .setNumTasks(10)
- .setVirtualCores(4)
- .setMemoryMb(1024)
- .setJavaOpts("")
- .setTaskModule("M7.class")
- .build()
- )
- .addOutEdgeId("M7_M5")
- .build()
- )
- .addVertex(VertexPlan.newBuilder()
- .setName("R6")
- .setProcessorDescriptor(
- TezEntityDescriptorProto.newBuilder().setClassName("R6.class"))
- .setType(PlanVertexType.NORMAL)
- .addTaskLocationHint(
- PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
- .setTaskConfig(PlanTaskConfiguration.newBuilder()
- .setNumTasks(1)
- .setVirtualCores(4)
- .setMemoryMb(1024)
- .setJavaOpts("")
- .setTaskModule("R6.class")
- .build()
- )
- .addInEdgeId("M5_R6")
- .build()
- )
- .addEdge(
- EdgePlan.newBuilder()
- .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3"))
- .setInputVertexName("M2")
- .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3.class"))
- .setOutputVertexName("R3")
- .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
- .setId("M2_R3")
- .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
- .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
- .build()
- )
- .addEdge(
- EdgePlan.newBuilder()
- .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5"))
- .setInputVertexName("R3")
- .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5.class"))
- .setOutputVertexName("M5")
- .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
- .setId("R3_M5")
- .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
- .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
- .build()
- )
- .addEdge(
- EdgePlan.newBuilder()
- .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5"))
- .setInputVertexName("M7")
- .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5.class"))
- .setOutputVertexName("M5")
- .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
- .setId("M7_M5")
- .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
- .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
- .build()
- )
- .addEdge(
- EdgePlan.newBuilder()
- .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6"))
- .setInputVertexName("M5")
- .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6.class"))
- .setOutputVertexName("R6")
- .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
- .setId("M5_R6")
- .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
- .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
- .build()
- )
- .addEdge(
- EdgePlan.newBuilder()
- .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5"))
- .setInputVertexName("M8")
- .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5.class"))
- .setEdgeManager(
- TezEntityDescriptorProto.newBuilder()
- .setClassName(EdgeManagerForTest.class.getName())
- .setUserPayload(ByteString.copyFrom(edgePayload))
- .build())
- .setOutputVertexName("M5")
- .setDataMovementType(PlanEdgeDataMovementType.CUSTOM)
- .setId("M8_M5")
- .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
- .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
- .build()
- )
- .build();
-
- return dag;
- }
-
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
public void testVertexWithInitializerSuccess() {
@@ -3516,7 +3215,6 @@ public class TestVertexImpl {
dispatcher.getEventHandler(), taskAttemptListener,
clock, thh, true, appContext, vertexLocationHint, null, taskSpecificLaunchCmdOption,
updateTracker);
- v.setInputVertices(new HashMap());
vertexIdMap.put(vId, v);
vertices.put(v.getName(), v);
v.handle(new VertexEvent(vId, VertexEventType.V_INIT));
@@ -3756,8 +3454,8 @@ public class TestVertexImpl {
List<GroupInputSpec> groupInSpec = vC.getGroupInputSpecList(0);
Assert.assertEquals(1, groupInSpec.size());
Assert.assertEquals("Group", groupInSpec.get(0).getGroupName());
- assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
- assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
+ Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
+ Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
groupInSpec.get(0).getMergedInputDescriptor().getClassName().equals("Group.class");
}
[19/25] git commit: TEZ-1563. TezClient.submitDAGSession alters DAG
local resources regardless of DAG submission (bikas)
Posted by je...@apache.org.
TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless of DAG submission (bikas)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/5d6b8fd5
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/5d6b8fd5
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/5d6b8fd5
Branch: refs/heads/TEZ-8
Commit: 5d6b8fd5660dbe5828467d5678bbbd58c3702c7a
Parents: 7d1303f
Author: Bikas Saha <bi...@apache.org>
Authored: Mon Sep 15 11:51:02 2014 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Mon Sep 15 11:51:02 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 4 +
.../java/org/apache/tez/client/TezClient.java | 5 +-
.../org/apache/tez/client/TezClientUtils.java | 15 +++-
.../main/java/org/apache/tez/dag/api/DAG.java | 9 ++-
.../org/apache/tez/client/TestTezClient.java | 77 +++++++++++++-------
.../org/apache/tez/dag/app/TestPreemption.java | 4 +-
6 files changed, 80 insertions(+), 34 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3198323..1520e70 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,6 +19,8 @@ ALL CHANGES:
TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
TEZ-1524. Resolve user group information only if ACLs are enabled.
TEZ-1581. GroupByOrderByMRRTest no longer functional.
+ TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
+ of DAG submission
Release 0.5.1: Unreleased
@@ -36,6 +38,8 @@ ALL CHANGES
TEZ-1519. TezTaskRunner should not initialize TezConfiguration in TezChild.
TEZ-1534. Make client side configs available to AM and tasks.
TEZ-1574. Support additional formats for the tez deployed archive
+ TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
+ of DAG submission
Release 0.5.0: 2014-09-03
http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index 77ab20c..5cec8b0 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -366,10 +366,9 @@ public class TezClient {
}
Map<String, LocalResource> tezJarResources = getTezJarResources(sessionCredentials);
- TezClientUtils.updateDAGVertices(dag, amConfig, tezJarResources,
+ DAGPlan dagPlan = TezClientUtils.prepareAndCreateDAGPlan(dag, amConfig, tezJarResources,
TezClientUtils.usingTezLibsFromArchive(tezJarResources), sessionCredentials);
-
- DAGPlan dagPlan = dag.createDag(amConfig.getTezConfiguration());
+
SubmitDAGRequestProto.Builder requestBuilder = SubmitDAGRequestProto.newBuilder();
requestBuilder.setDAGPlan(dagPlan).build();
if (!additionalLocalResources.isEmpty()) {
http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 3297adc..917fcff 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -557,7 +557,7 @@ public class TezClientUtils {
if(dag != null) {
- updateDAGVertices(dag, amConfig, tezJarResources, tezLrsAsArchive, sessionCreds);
+ DAGPlan dagPB = prepareAndCreateDAGPlan(dag, amConfig, tezJarResources, tezLrsAsArchive, sessionCreds);
// emit protobuf DAG file style
Path binaryPath = TezCommonUtils.getTezBinPlanStagingPath(tezSysStagingPath);
@@ -567,8 +567,6 @@ public class TezClientUtils {
+ sessionJarsPath + " binaryPlanPath :" + binaryPath);
}
- DAGPlan dagPB = dag.createDag(amConfig.getTezConfiguration());
-
FSDataOutputStream dagPBOutBinaryStream = null;
try {
@@ -644,6 +642,17 @@ public class TezClientUtils {
}
}
+ static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig,
+ Map<String, LocalResource> tezJarResources, boolean tezLrsAsArchive,
+ Credentials credentials) throws IOException {
+ DAGPlan dagPB = dag.getCachedDAGPlan();
+ if (dagPB == null) {
+ updateDAGVertices(dag, amConfig, tezJarResources, tezLrsAsArchive, credentials);
+ dagPB = dag.createDag(amConfig.getTezConfiguration());
+ }
+ return dagPB;
+ }
+
static void maybeAddDefaultLoggingJavaOpts(String logLevel, List<String> vargs) {
if (vargs != null && !vargs.isEmpty()) {
for (String arg : vargs) {
http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
index d063e79..ffd2e83 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -89,6 +89,7 @@ public class DAG {
Map<String, LocalResource> commonTaskLocalFiles = Maps.newHashMap();
private Stack<String> topologicalVertexStack = new Stack<String>();
+ private DAGPlan cachedDAGPlan;
private DAG(String name) {
this.name = name;
@@ -574,6 +575,10 @@ public class DAG {
}
}
+ @Private
+ public DAGPlan getCachedDAGPlan() {
+ return cachedDAGPlan;
+ }
// create protobuf message describing DAG
@Private
@@ -787,6 +792,8 @@ public class DAG {
dagBuilder.setCredentialsBinary(DagTypeConverters.convertCredentialsToProto(credentials));
TezCommonUtils.logCredentials(LOG, credentials, "dag");
}
- return dagBuilder.build();
+
+ cachedDAGPlan = dagBuilder.build();
+ return cachedDAGPlan;
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
index 5044d55..fbb4486 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
@@ -20,6 +20,7 @@ package org.apache.tez.client;
import java.io.IOException;
import java.util.Collections;
+import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
@@ -89,7 +90,16 @@ public class TestTezClient {
}
}
- static void configure(TezClientForTest client) throws YarnException, IOException {
+ TezClientForTest configure() throws YarnException, IOException {
+ return configure(new HashMap<String, LocalResource>(), true);
+ }
+
+ TezClientForTest configure(Map<String, LocalResource> lrs, boolean isSession) throws YarnException, IOException {
+ TezConfiguration conf = new TezConfiguration();
+ conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
+ conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, isSession);
+ TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
+
ApplicationId appId1 = ApplicationId.newInstance(0, 1);
YarnClient yarnClient = mock(YarnClient.class, RETURNS_DEEP_STUBS);
when(yarnClient.createApplication().getNewApplicationResponse().getApplicationId()).thenReturn(appId1);
@@ -100,31 +110,27 @@ public class TestTezClient {
client.mockTezYarnClient = new TezYarnClient(yarnClient);
client.mockYarnClient = yarnClient;
client.mockAppId = appId1;
+
+ return client;
}
- @Test
+ @Test (timeout = 5000)
public void testTezclientApp() throws Exception {
testTezClient(false);
}
- @Test
+ @Test (timeout = 5000)
public void testTezclientSession() throws Exception {
testTezClient(true);
}
public void testTezClient(boolean isSession) throws Exception {
- TezConfiguration conf = new TezConfiguration();
- conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
- conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, isSession);
-
Map<String, LocalResource> lrs = Maps.newHashMap();
String lrName1 = "LR1";
lrs.put(lrName1, LocalResource.newInstance(URL.newInstance("file:///", "localhost", 0, "test"),
LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
- TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
-
- configure(client);
+ TezClientForTest client = configure(lrs, isSession);
ArgumentCaptor<ApplicationSubmissionContext> captor = ArgumentCaptor.forClass(ApplicationSubmissionContext.class);
client.start();
@@ -229,13 +235,7 @@ public class TestTezClient {
}
public void testPreWarm() throws Exception {
- TezConfiguration conf = new TezConfiguration();
- conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
- conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
- Map<String, LocalResource> lrs = Maps.newHashMap();
- final TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
-
- configure(client);
+ TezClientForTest client = configure();
client.start();
PreWarmVertex vertex = PreWarmVertex.create("PreWarm", 1, Resource.newInstance(1, 1));
client.preWarm(vertex);
@@ -248,16 +248,42 @@ public class TestTezClient {
client.stop();
}
+ @Test (timeout = 10000)
+ public void testMultipleSubmissions() throws Exception {
+ testMultipleSubmissionsJob(false);
+ testMultipleSubmissionsJob(true);
+ }
+
+ public void testMultipleSubmissionsJob(boolean isSession) throws Exception {
+ TezClientForTest client1 = configure(new HashMap<String, LocalResource>(), isSession);
+ client1.start();
+
+ String mockLR1Name = "LR1";
+ Map<String, LocalResource> lrDAG = Collections.singletonMap(mockLR1Name, LocalResource
+ .newInstance(URL.newInstance("file:///", "localhost", 0, "test"), LocalResourceType.FILE,
+ LocalResourceVisibility.PUBLIC, 1, 1));
+ Vertex vertex = Vertex.create("Vertex", ProcessorDescriptor.create("P"), 1,
+ Resource.newInstance(1, 1));
+ DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG);
+ // the dag resource will be added to the vertex once
+ client1.submitDAG(dag);
+
+ TezClientForTest client2 = configure();
+ client2.start();
+
+ // verify resubmission of same dag to new client (simulates submission error resulting in the
+ // creation of a new client and resubmission of the DAG)
+ // TEZ-1563 dag resource will not be added again to the vertex because its cached
+ // so resubmission works fine
+ client2.submitDAG(dag);
+
+ client1.stop();
+ client2.stop();
+ }
+
@Test(timeout = 5000)
public void testWaitTillReady_Interrupt() throws Exception {
- TezConfiguration conf = new TezConfiguration();
- conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
- conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
- Map<String, LocalResource> lrs = Maps.newHashMap();
- final TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
-
- configure(client);
-
+ final TezClientForTest client = configure();
client.start();
when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState())
@@ -278,6 +304,7 @@ public class TestTezClient {
thread.interrupt();
thread.join();
Assert.assertThat(exceptionReference.get(),CoreMatchers. instanceOf(InterruptedException.class));
+ client.stop();
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
index c7aacd4..cfb1c9f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
@@ -85,7 +85,7 @@ public class TestPreemption {
return dag;
}
- @Test
+ @Test (timeout = 5000)
public void testPreemptionWithoutSession() throws Exception {
System.out.println("TestPreemptionWithoutSession");
TezConfiguration tezconf = new TezConfiguration(defaultConf);
@@ -120,7 +120,7 @@ public class TestPreemption {
tezClient.stop();
}
- @Test
+ @Test (timeout = 30000)
public void testPreemptionWithSession() throws Exception {
System.out.println("TestPreemptionWithSession");
MockTezClient tezClient = createTezSession();
[18/25] git commit: TEZ-1581. GroupByOrderByMRRTest no longer
functional. (hitesh)
Posted by je...@apache.org.
TEZ-1581. GroupByOrderByMRRTest no longer functional. (hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/7d1303fa
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/7d1303fa
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/7d1303fa
Branch: refs/heads/TEZ-8
Commit: 7d1303fa606e700bd4d4b2a122a73a5badbbc889
Parents: edb841c
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 15:18:05 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 15:18:05 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../examples/GroupByOrderByMRRTest.java | 283 +++++++++++++------
2 files changed, 205 insertions(+), 79 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/7d1303fa/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 59be260..3198323 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -18,6 +18,7 @@ ALL CHANGES:
TEZ-1569. Add tests for preemption
TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
TEZ-1524. Resolve user group information only if ACLs are enabled.
+ TEZ-1581. GroupByOrderByMRRTest no longer functional.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/7d1303fa/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
index 939bea0..393faea 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
@@ -18,34 +18,54 @@
package org.apache.tez.mapreduce.examples;
+import java.io.ByteArrayOutputStream;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
import java.util.StringTokenizer;
+import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.client.MRTezClient;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.DataSourceDescriptor;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.Vertex;
import org.apache.tez.dag.api.client.DAGClient;
import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
+import org.apache.tez.mapreduce.output.MROutputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig;
+import org.apache.tez.runtime.library.partitioner.HashPartitioner;
/**
* Simple example that does a GROUP BY ORDER BY in an MRR job
@@ -94,7 +114,7 @@ public class GroupByOrderByMRRTest extends Configured implements Tool {
public void map(Object key, Text value, Context context
) throws IOException, InterruptedException {
StringTokenizer itr = new StringTokenizer(value.toString());
- String empName = "";
+ String empName;
String deptName = "";
if (itr.hasMoreTokens()) {
empName = itr.nextToken();
@@ -149,25 +169,141 @@ public class GroupByOrderByMRRTest extends Configured implements Tool {
}
}
+ private static DAG createDAG(Configuration conf, Map<String, LocalResource> commonLocalResources,
+ Path stagingDir, String inputPath, String outputPath, boolean useMRSettings)
+ throws Exception {
+
+ Configuration mapStageConf = new JobConf(conf);
+ mapStageConf.set(MRJobConfig.MAP_CLASS_ATTR,
+ MyMapper.class.getName());
+
+ MRHelpers.translateMRConfToTez(mapStageConf);
+
+ Configuration iReduceStageConf = new JobConf(conf);
+ // TODO replace with auto-reduce parallelism
+ iReduceStageConf.setInt(MRJobConfig.NUM_REDUCES, 2);
+ iReduceStageConf.set(MRJobConfig.REDUCE_CLASS_ATTR,
+ MyGroupByReducer.class.getName());
+ iReduceStageConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, Text.class.getName());
+ iReduceStageConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS,
+ IntWritable.class.getName());
+ iReduceStageConf.setBoolean("mapred.mapper.new-api", true);
+ MRHelpers.translateMRConfToTez(iReduceStageConf);
+
+ Configuration finalReduceConf = new JobConf(conf);
+ finalReduceConf.setInt(MRJobConfig.NUM_REDUCES, 1);
+ finalReduceConf.set(MRJobConfig.REDUCE_CLASS_ATTR,
+ MyOrderByNoOpReducer.class.getName());
+ finalReduceConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, IntWritable.class.getName());
+ finalReduceConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, Text.class.getName());
+ MRHelpers.translateMRConfToTez(finalReduceConf);
+
+ MRHelpers.configureMRApiUsage(mapStageConf);
+ MRHelpers.configureMRApiUsage(iReduceStageConf);
+ MRHelpers.configureMRApiUsage(finalReduceConf);
+
+ List<Vertex> vertices = new ArrayList<Vertex>();
+
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream(4096);
+ mapStageConf.writeXml(outputStream);
+ String mapStageHistoryText = new String(outputStream.toByteArray(), "UTF-8");
+ mapStageConf.set(MRJobConfig.INPUT_FORMAT_CLASS_ATTR,
+ TextInputFormat.class.getName());
+ mapStageConf.set(FileInputFormat.INPUT_DIR, inputPath);
+ mapStageConf.setBoolean("mapred.mapper.new-api", true);
+ DataSourceDescriptor dsd = MRInputHelpers.configureMRInputWithLegacySplitGeneration(
+ mapStageConf, stagingDir, true);
+
+ Vertex mapVertex;
+ ProcessorDescriptor mapProcessorDescriptor =
+ ProcessorDescriptor.create(MapProcessor.class.getName())
+ .setUserPayload(
+ TezUtils.createUserPayloadFromConf(mapStageConf))
+ .setHistoryText(mapStageHistoryText);
+ if (!useMRSettings) {
+ mapVertex = Vertex.create("initialmap", mapProcessorDescriptor);
+ } else {
+ mapVertex = Vertex.create("initialmap", mapProcessorDescriptor, -1,
+ MRHelpers.getResourceForMRMapper(mapStageConf));
+ mapVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
+ }
+ mapVertex.addTaskLocalFiles(commonLocalResources)
+ .addDataSource("MRInput", dsd);
+ vertices.add(mapVertex);
+
+ ByteArrayOutputStream iROutputStream = new ByteArrayOutputStream(4096);
+ iReduceStageConf.writeXml(iROutputStream);
+ String iReduceStageHistoryText = new String(iROutputStream.toByteArray(), "UTF-8");
+
+ ProcessorDescriptor iReduceProcessorDescriptor = ProcessorDescriptor.create(
+ ReduceProcessor.class.getName())
+ .setUserPayload(TezUtils.createUserPayloadFromConf(iReduceStageConf))
+ .setHistoryText(iReduceStageHistoryText);
+
+ Vertex intermediateVertex;
+ if (!useMRSettings) {
+ intermediateVertex = Vertex.create("ireduce1", iReduceProcessorDescriptor, 1);
+ } else {
+ intermediateVertex = Vertex.create("ireduce1", iReduceProcessorDescriptor,
+ 1, MRHelpers.getResourceForMRReducer(iReduceStageConf));
+ intermediateVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(iReduceStageConf));
+ }
+ intermediateVertex.addTaskLocalFiles(commonLocalResources);
+ vertices.add(intermediateVertex);
+
+ ByteArrayOutputStream finalReduceOutputStream = new ByteArrayOutputStream(4096);
+ finalReduceConf.writeXml(finalReduceOutputStream);
+ String finalReduceStageHistoryText = new String(finalReduceOutputStream.toByteArray(), "UTF-8");
+ UserPayload finalReducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
+ Vertex finalReduceVertex;
+
+ ProcessorDescriptor finalReduceProcessorDescriptor =
+ ProcessorDescriptor.create(
+ ReduceProcessor.class.getName())
+ .setUserPayload(finalReducePayload)
+ .setHistoryText(finalReduceStageHistoryText);
+ if (!useMRSettings) {
+ finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1);
+ } else {
+ finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1,
+ MRHelpers.getResourceForMRReducer(finalReduceConf));
+ finalReduceVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
+ }
+ finalReduceVertex.addTaskLocalFiles(commonLocalResources);
+ finalReduceVertex.addDataSink("MROutput",
+ MROutputLegacy.createConfigBuilder(finalReduceConf, TextOutputFormat.class, outputPath)
+ .build());
+ vertices.add(finalReduceVertex);
+
+ DAG dag = DAG.create("groupbyorderbymrrtest");
+ for (Vertex v : vertices) {
+ dag.addVertex(v);
+ }
+
+ OrderedPartitionedKVEdgeConfig edgeConf1 = OrderedPartitionedKVEdgeConfig
+ .newBuilder(Text.class.getName(), IntWritable.class.getName(),
+ HashPartitioner.class.getName()).setFromConfiguration(conf)
+ .configureInput().useLegacyInput().done().build();
+ dag.addEdge(
+ Edge.create(dag.getVertex("initialmap"), dag.getVertex("ireduce1"),
+ edgeConf1.createDefaultEdgeProperty()));
+
+ OrderedPartitionedKVEdgeConfig edgeConf2 = OrderedPartitionedKVEdgeConfig
+ .newBuilder(IntWritable.class.getName(), Text.class.getName(),
+ HashPartitioner.class.getName()).setFromConfiguration(conf)
+ .configureInput().useLegacyInput().done().build();
+ dag.addEdge(
+ Edge.create(dag.getVertex("ireduce1"), dag.getVertex("finalreduce"),
+ edgeConf2.createDefaultEdgeProperty()));
+
+ return dag;
+ }
+
+
@Override
public int run(String[] args) throws Exception {
Configuration conf = getConf();
- // Configure intermediate reduces
- conf.setInt(MRJobConfig.MRR_INTERMEDIATE_STAGES, 1);
-
- // Set reducer class for intermediate reduce
- conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
- "mapreduce.job.reduce.class"), MyGroupByReducer.class, Reducer.class);
- // Set reducer output key class
- conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
- "mapreduce.map.output.key.class"), IntWritable.class, Object.class);
- // Set reducer output value class
- conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
- "mapreduce.map.output.value.class"), Text.class, Object.class);
- conf.setInt(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
- "mapreduce.job.reduces"), 2);
-
String[] otherArgs = new GenericOptionsParser(conf, args).
getRemainingArgs();
if (otherArgs.length != 2) {
@@ -176,66 +312,55 @@ public class GroupByOrderByMRRTest extends Configured implements Tool {
return 2;
}
- @SuppressWarnings("deprecation")
- Job job = new Job(conf, "groupbyorderbymrrtest");
-
- job.setJarByClass(GroupByOrderByMRRTest.class);
-
- // Configure map
- job.setMapperClass(MyMapper.class);
- job.setMapOutputKeyClass(Text.class);
- job.setMapOutputValueClass(IntWritable.class);
-
- // Configure reduce
- job.setReducerClass(MyOrderByNoOpReducer.class);
- job.setOutputKeyClass(Text.class);
- job.setOutputValueClass(IntWritable.class);
- job.setNumReduceTasks(1);
-
- FileInputFormat.addInputPath(job, new Path(otherArgs[0]));
- FileOutputFormat.setOutputPath(job, new Path(otherArgs[1]));
-
- job.submit();
- JobID jobId = job.getJobID();
- ApplicationId appId = TypeConverter.toYarn(jobId).getAppId();
-
- DAGClient dagClient = MRTezClient.getDAGClient(appId, new TezConfiguration(conf), null);
- DAGStatus dagStatus;
- String[] vNames = { "initialmap" , "ireduce1" , "finalreduce" };
- while (true) {
- dagStatus = dagClient.getDAGStatus(null);
- if(dagStatus.getState() == DAGStatus.State.RUNNING ||
- dagStatus.getState() == DAGStatus.State.SUCCEEDED ||
- dagStatus.getState() == DAGStatus.State.FAILED ||
- dagStatus.getState() == DAGStatus.State.KILLED ||
- dagStatus.getState() == DAGStatus.State.ERROR) {
- break;
- }
- try {
- Thread.sleep(500);
- } catch (InterruptedException e) {
- // continue;
- }
+ String inputPath = otherArgs[0];
+ String outputPath = otherArgs[1];
+
+ UserGroupInformation.setConfiguration(conf);
+
+ TezConfiguration tezConf = new TezConfiguration(conf);
+ FileSystem fs = FileSystem.get(conf);
+
+ if (fs.exists(new Path(outputPath))) {
+ throw new FileAlreadyExistsException("Output directory "
+ + outputPath + " already exists");
}
- while (dagStatus.getState() == DAGStatus.State.RUNNING) {
- try {
- ExampleDriver.printDAGStatus(dagClient, vNames);
- try {
- Thread.sleep(1000);
- } catch (InterruptedException e) {
- // continue;
- }
- dagStatus = dagClient.getDAGStatus(null);
- } catch (TezException e) {
- LOG.fatal("Failed to get application progress. Exiting");
+ Map<String, LocalResource> localResources =
+ new TreeMap<String, LocalResource>();
+
+ String stagingDirStr = conf.get(TezConfiguration.TEZ_AM_STAGING_DIR,
+ TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT) + Path.SEPARATOR +
+ Long.toString(System.currentTimeMillis());
+ Path stagingDir = new Path(stagingDirStr);
+ FileSystem pathFs = stagingDir.getFileSystem(tezConf);
+ pathFs.mkdirs(new Path(stagingDirStr));
+
+ tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirStr);
+ stagingDir = pathFs.makeQualified(new Path(stagingDirStr));
+
+ TezClient tezClient = TezClient.create("groupbyorderbymrrtest", tezConf);
+ tezClient.start();
+
+ LOG.info("Submitting groupbyorderbymrrtest DAG as a new Tez Application");
+
+ try {
+ DAG dag = createDAG(conf, localResources, stagingDir, inputPath, outputPath, true);
+
+ tezClient.waitTillReady();
+
+ DAGClient dagClient = tezClient.submitDAG(dag);
+
+ DAGStatus dagStatus = dagClient.waitForCompletionWithStatusUpdates(null);
+ if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) {
+ LOG.error("groupbyorderbymrrtest failed, state=" + dagStatus.getState()
+ + ", diagnostics=" + dagStatus.getDiagnostics());
return -1;
}
+ LOG.info("Application completed. " + "FinalState=" + dagStatus.getState());
+ return 0;
+ } finally {
+ tezClient.stop();
}
-
- ExampleDriver.printDAGStatus(dagClient, vNames);
- LOG.info("Application completed. " + "FinalState=" + dagStatus.getState());
- return dagStatus.getState() == DAGStatus.State.SUCCEEDED ? 0 : 1;
}
public static void main(String[] args) throws Exception {
[21/25] git commit: TEZ-1585. Memory leak in tez session mode (Rajesh
Balamohan)
Posted by je...@apache.org.
TEZ-1585. Memory leak in tez session mode (Rajesh Balamohan)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/938097d8
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/938097d8
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/938097d8
Branch: refs/heads/TEZ-8
Commit: 938097d85f6f943a399c16fa50f07a7e2b05bdb5
Parents: dbe6a38
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Tue Sep 16 11:51:13 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Tue Sep 16 11:51:13 2014 +0530
----------------------------------------------------------------------
.../src/main/java/org/apache/tez/dag/app/DAGAppMaster.java | 7 +++++++
1 file changed, 7 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/938097d8/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 ea0ab3b..0e4f78b 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
@@ -536,6 +536,13 @@ public class DAGAppMaster extends AbstractService {
}
}
}
+ //close all fs related caches
+ try {
+ FileSystem.closeAllForUGI(context.getCurrentDAG().getDagUGI());
+ } catch (IOException e) {
+ LOG.warn("Error occurred when trying to close FileSystem for userName " + context
+ .getCurrentDAG().getDagUGI().getUserName(), e);
+ }
break;
case AM_REBOOT:
LOG.info("Received an AM_REBOOT signal");
[09/25] git commit: TEZ-1539. Change InputInitializerEvent semantics
to SEND_ONCE_ON_TASK_SUCCESS. (sseth)
Posted by je...@apache.org.
TEZ-1539. Change InputInitializerEvent semantics to
SEND_ONCE_ON_TASK_SUCCESS. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b4580a7b
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b4580a7b
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b4580a7b
Branch: refs/heads/TEZ-8
Commit: b4580a7b81ab1a5619987296641ebbf50fda4c55
Parents: fb05aac
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 12 02:37:11 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 12 02:37:11 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 4 +
.../tez/runtime/api/InputInitializer.java | 5 +-
.../api/events/InputInitializerEvent.java | 23 +
.../org/apache/tez/dag/app/RecoveryParser.java | 8 +-
.../app/dag/RootInputInitializerManager.java | 194 +++++-
.../tez/dag/app/dag/StateChangeNotifier.java | 76 ++-
.../dag/app/dag/TaskStateUpdateListener.java | 35 +
.../apache/tez/dag/app/dag/impl/TaskImpl.java | 45 +-
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 75 ++-
.../VertexDataMovementEventsGeneratedEvent.java | 214 ------
.../VertexRecoverableEventsGeneratedEvent.java | 223 +++++++
tez-dag/src/main/proto/HistoryEvents.proto | 1 +
.../tez/dag/app/dag/impl/TestTaskImpl.java | 10 +-
.../tez/dag/app/dag/impl/TestTaskRecovery.java | 3 +-
.../tez/dag/app/dag/impl/TestVertexImpl.java | 648 ++++++++++++++++++-
.../dag/app/dag/impl/TestVertexRecovery.java | 16 +-
.../TestHistoryEventsProtoConversion.java | 10 +-
.../impl/TestHistoryEventJsonConversion.java | 4 +-
.../ats/TestHistoryEventTimelineConversion.java | 4 +-
.../org/apache/tez/test/TestDAGRecovery.java | 6 +-
20 files changed, 1290 insertions(+), 314 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f29d48d..dd20f43 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,10 @@ ALL CHANGES:
Release 0.5.1: Unreleased
+INCOMPATIBLE CHANGES
+ TEZ-1539. Change InputInitializerEvent semantics to SEND_ONCE_ON_TASK_SUCCESS
+ TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
+
ALL CHANGES
TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput()
TEZ-1515. Remove usage of ResourceBundles in Counters.
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
index 00896de..7b22b62 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
@@ -93,7 +93,10 @@ public abstract class InputInitializer {
* State changes will be received based on the registration via {@link
* org.apache.tez.runtime.api.InputInitializerContext#registerForVertexStateUpdates(String,
* java.util.Set)}. Notifications will be received for all registered state changes, and not just
- * for the latest state update. They will be in order in which the state change occurred.
+ * for the latest state update. They will be in order in which the state change occurred. </p>
+ *
+ * Extensive processing should not be performed via this method call. Instead this should just be
+ * used as a notification mechanism to the main initialization, which is via the initialize method.
*
* @param stateUpdate an event indicating the name of the vertex, and it's updated state.
* Additional information may be available for specific events, Look at the
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
index 3c5e78e..8360447 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.tez.runtime.api.InputInitializer;
@@ -32,6 +33,13 @@ import org.apache.tez.runtime.api.Event;
/**
* An event that is routed to the specified {@link InputInitializer}.
* This can be used to send information/metadata to the {@link InputInitializer}
+ * <p/>
+ *
+ * These events are routed to the InputInitializer, only after the task which generated the event
+ * succeeds. Also, the events will only be sent once per task - irrespective of how many attempts
+ * were run, or succeeded. An example of this is when a task is retried because the node on which it
+ * was running failed. If the Task had succeeded once, the event would already have been sent - and
+ * will not be resent when the task reruns and succeeds. </p>
*/
@Unstable
@Public
@@ -41,6 +49,7 @@ public class InputInitializerEvent extends Event {
private String targetInputName;
private ByteBuffer eventPayload;
+ private String sourceVertexName;
private InputInitializerEvent(String targetVertexName, String targetInputName,
ByteBuffer eventPayload) {
@@ -88,4 +97,18 @@ public class InputInitializerEvent extends Event {
public ByteBuffer getUserPayload() {
return eventPayload == null ? null : eventPayload.asReadOnlyBuffer();
}
+
+ @InterfaceAudience.Private
+ public void setSourceVertexName(String srcVertexName) {
+ this.sourceVertexName = srcVertexName;
+ }
+
+ /**
+ * Returns the name of the vertex which generated the event. This will only be populated after
+ * the event has been routed by the AM.
+ * @return the name of the source vertex
+ */
+ public String getSourceVertexName() {
+ return this.sourceVertexName;
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
index 9ba5847..85851c5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
@@ -58,7 +58,7 @@ import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
import org.apache.tez.dag.history.events.TaskFinishedEvent;
import org.apache.tez.dag.history.events.TaskStartedEvent;
import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
import org.apache.tez.dag.history.events.VertexFinishedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
@@ -207,7 +207,7 @@ public class RecoveryParser {
event = new TaskAttemptFinishedEvent();
break;
case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
- event = new VertexDataMovementEventsGeneratedEvent();
+ event = new VertexRecoverableEventsGeneratedEvent();
break;
default:
throw new IOException("Invalid data found, unknown event type "
@@ -865,8 +865,8 @@ public class RecoveryParser {
+ ", eventType=" + eventType
+ ", event=" + event.toString());
assert recoveredDAGData.recoveredDAG != null;
- VertexDataMovementEventsGeneratedEvent vEvent =
- (VertexDataMovementEventsGeneratedEvent) event;
+ VertexRecoverableEventsGeneratedEvent vEvent =
+ (VertexRecoverableEventsGeneratedEvent) event;
Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
v.restoreFromEvent(vEvent);
break;
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
index 770761e..87d4eb6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
@@ -20,7 +20,10 @@ package org.apache.tez.dag.app.dag;
import javax.annotation.Nullable;
import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -30,10 +33,13 @@ import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.base.Preconditions;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
@@ -43,10 +49,12 @@ import org.apache.tez.dag.api.RootInputLeafOutput;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.event.*;
import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.oldrecords.TaskState;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.dag.event.VertexEventRootInputFailed;
import org.apache.tez.dag.app.dag.event.VertexEventRootInputInitialized;
import org.apache.tez.dag.app.dag.impl.TezRootInputInitializerContextImpl;
+import org.apache.tez.dag.records.TezTaskID;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.runtime.api.Event;
import org.apache.tez.runtime.api.InputInitializer;
@@ -61,6 +69,7 @@ import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
public class RootInputInitializerManager {
@@ -77,7 +86,8 @@ public class RootInputInitializerManager {
private final Vertex vertex;
private final AppContext appContext;
- private final Map<String, InitializerWrapper> initializerMap = new HashMap<String, InitializerWrapper>();
+ @VisibleForTesting
+ final Map<String, InitializerWrapper> initializerMap = new HashMap<String, InitializerWrapper>();
public RootInputInitializerManager(Vertex vertex, AppContext appContext,
UserGroupInformation dagUgi, StateChangeNotifier stateTracker) {
@@ -100,7 +110,7 @@ public class RootInputInitializerManager {
InputInitializer initializer = createInitializer(input, context);
InitializerWrapper initializerWrapper =
- new InitializerWrapper(input, initializer, context, vertex, entityStateTracker);
+ new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext);
initializerMap.put(input.getName(), initializerWrapper);
ListenableFuture<List<Event>> future = executor
.submit(new InputInitializerCallable(initializerWrapper, dagUgi));
@@ -117,31 +127,40 @@ public class RootInputInitializerManager {
return initializer;
}
- public void handleInitializerEvent(InputInitializerEvent event) {
- Preconditions.checkState(vertex.getName().equals(event.getTargetVertexName()),
- "Received event for incorrect vertex");
- Preconditions.checkNotNull(event.getTargetInputName(), "target input name must be set");
- InitializerWrapper initializer = initializerMap.get(event.getTargetInputName());
- Preconditions.checkState(initializer != null,
- "Received event for unknown input : " + event.getTargetInputName());
+ public void handleInitializerEvents(List<TezEvent> events) {
+ ListMultimap<InitializerWrapper, TezEvent> eventMap = LinkedListMultimap.create();
+
+ for (TezEvent tezEvent : events) {
+ Preconditions.checkState(tezEvent.getEvent() instanceof InputInitializerEvent);
+ InputInitializerEvent event = (InputInitializerEvent)tezEvent.getEvent();
+ Preconditions.checkState(vertex.getName().equals(event.getTargetVertexName()),
+ "Received event for incorrect vertex");
+ Preconditions.checkNotNull(event.getTargetInputName(), "target input name must be set");
+ InitializerWrapper initializer = initializerMap.get(event.getTargetInputName());
+ Preconditions.checkState(initializer != null,
+ "Received event for unknown input : " + event.getTargetInputName());
+ eventMap.put(initializer, tezEvent);
+ }
+
// This is a restriction based on current flow - i.e. events generated only by initialize().
// TODO Rework the flow as per the first comment on TEZ-1076
if (isStopped) {
LOG.warn("InitializerManager already stopped for " + vertex.getLogIdentifier() +
- " Dropping event. [" + event + "]");
- return;
+ " Dropping " + events.size() + " events");
}
- if (initializer.isComplete()) {
- LOG.warn(
- "Event targeted at vertex " + vertex.getLogIdentifier() + ", initializerWrapper for Input: " +
- initializer.getInput().getName() +
- " will be dropped, since Input has already been initialized. [" + event + "]");
- }
- try {
- initializer.getInitializer().handleInputInitializerEvent(Lists.newArrayList(event));
- } catch (Exception e) {
- throw new TezUncheckedException(
- "Initializer for input: " + event.getTargetInputName() + " failed to process event", e);
+
+ for (Map.Entry<InitializerWrapper, Collection<TezEvent>> entry : eventMap.asMap().entrySet()) {
+ InitializerWrapper initializerWrapper = entry.getKey();
+ if (initializerWrapper.isComplete()) {
+ LOG.warn(entry.getValue().size() +
+ " events targeted at vertex " + vertex.getLogIdentifier() +
+ ", initializerWrapper for Input: " +
+ initializerWrapper.getInput().getName() +
+ " will be dropped, since Input has already been initialized.");
+ } else {
+ initializerWrapper.handleInputInitializerEvents(entry.getValue());
+ }
+
}
}
@@ -157,7 +176,13 @@ public class RootInputInitializerManager {
protected InputInitializerCallback createInputInitializerCallback(InitializerWrapper initializer) {
return new InputInitializerCallback(initializer, eventHandler, vertex.getVertexId());
}
-
+
+ @VisibleForTesting
+ @InterfaceAudience.Private
+ public InitializerWrapper getInitializerWrapper(String inputName) {
+ return initializerMap.get(inputName);
+ }
+
public void shutdown() {
if (executor != null && !isStopped) {
// Don't really care about what is running if an error occurs. If no error
@@ -232,7 +257,9 @@ public class RootInputInitializerManager {
}
}
- private static class InitializerWrapper implements VertexStateUpdateListener {
+ @VisibleForTesting
+ @InterfaceAudience.Private
+ public static class InitializerWrapper implements VertexStateUpdateListener, TaskStateUpdateListener {
private final RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input;
@@ -242,15 +269,18 @@ public class RootInputInitializerManager {
private final String vertexLogIdentifier;
private final StateChangeNotifier stateChangeNotifier;
private final List<String> notificationRegisteredVertices = Lists.newArrayList();
+ private final AppContext appContext;
InitializerWrapper(RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input,
InputInitializer initializer, InputInitializerContext context,
- Vertex vertex, StateChangeNotifier stateChangeNotifier) {
+ Vertex vertex, StateChangeNotifier stateChangeNotifier,
+ AppContext appContext) {
this.input = input;
this.initializer = initializer;
this.context = context;
this.vertexLogIdentifier = vertex.getLogIdentifier();
this.stateChangeNotifier = stateChangeNotifier;
+ this.appContext = appContext;
}
public RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> getInput() {
@@ -272,6 +302,7 @@ public class RootInputInitializerManager {
public void setComplete() {
this.isComplete.set(true);
unregisterForVertexStatusUpdates();
+ unregisterForTaskStatusUpdates();
}
public void registerForVertexStateUpdates(String vertexName, Set<VertexState> stateSet) {
@@ -302,7 +333,118 @@ public class RootInputInitializerManager {
initializer.onVertexStateUpdated(event);
}
}
- }
+ private final Map<String, Map<Integer, Integer>> firstSuccessfulAttemptMap = new HashMap<String, Map<Integer, Integer>>();
+ private final ListMultimap<String, TezEvent> pendingEvents = LinkedListMultimap.create();
+ private final List<String> taskNotificationRegisteredVertices = Lists.newLinkedList();
+
+ @InterfaceAudience.Private
+ @VisibleForTesting
+ public Map<String, Map<Integer, Integer>> getFirstSuccessfulAttemptMap() {
+ return this.firstSuccessfulAttemptMap;
+ }
+
+ @InterfaceAudience.Private
+ @VisibleForTesting
+ public ListMultimap<String, TezEvent> getPendingEvents() {
+ return this.pendingEvents;
+ }
+
+ @Override
+ public void onTaskSucceeded(String vertexName, TezTaskID taskId, int attemptId) {
+ // Notifications will only start coming in after an event is received, which is when we register for notifications.
+ // TODO TEZ-1577. Get rid of this.
+ if (attemptId == -1) {
+ throw new TezUncheckedException(
+ "AttemptId is -1. This is likely caused by TEZ-1577; recovery not supported when InputInitializerEvents are used");
+ }
+ Map<Integer, Integer> vertexSuccessfulAttemptMap = firstSuccessfulAttemptMap.get(vertexName);
+ Integer successfulAttempt = vertexSuccessfulAttemptMap.get(taskId);
+ if (successfulAttempt == null) {
+ successfulAttempt = attemptId;
+ vertexSuccessfulAttemptMap.put(taskId.getId(), successfulAttempt);
+ }
+
+ // Run through all the pending events for this srcVertex to see if any of them need to be dispatched.
+ List<TezEvent> events = pendingEvents.get(vertexName);
+ if (events != null && !events.isEmpty()) {
+ List<InputInitializerEvent> toForwardEvents = new LinkedList<InputInitializerEvent>();
+ Iterator<TezEvent> eventIterator = events.iterator();
+ while (eventIterator.hasNext()) {
+ TezEvent tezEvent = eventIterator.next();
+ int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId();
+ int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId();
+ if (taskIndex == taskId.getId()) {
+ // Process only if there's a pending event for the specific succeeded task
+ if (taskAttemptIndex == successfulAttempt) {
+ toForwardEvents.add((InputInitializerEvent) tezEvent.getEvent());
+ }
+ eventIterator.remove();
+ }
+ }
+ sendEvents(toForwardEvents);
+ }
+ }
+
+ public void handleInputInitializerEvents(Collection<TezEvent> tezEvents) {
+ List<InputInitializerEvent> toForwardEvents = new LinkedList<InputInitializerEvent>();
+ for (TezEvent tezEvent : tezEvents) {
+ String srcVertexName = tezEvent.getSourceInfo().getTaskVertexName();
+ int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId();
+ int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId();
+
+ Map<Integer, Integer> vertexSuccessfulAttemptMap =
+ firstSuccessfulAttemptMap.get(srcVertexName);
+ if (vertexSuccessfulAttemptMap == null) {
+ vertexSuccessfulAttemptMap = new HashMap<Integer, Integer>();
+ firstSuccessfulAttemptMap.put(srcVertexName, vertexSuccessfulAttemptMap);
+ // Seen first time. Register for task updates
+ stateChangeNotifier.registerForTaskSuccessUpdates(srcVertexName, this);
+ taskNotificationRegisteredVertices.add(srcVertexName);
+ }
+
+ // Determine the successful attempt for the task
+ Integer successfulAttemptInteger = vertexSuccessfulAttemptMap.get(taskIndex);
+ if (successfulAttemptInteger == null) {
+ // Check immediately if this task has succeeded, in case the notification came in before the event
+ Vertex srcVertex = appContext.getCurrentDAG().getVertex(srcVertexName);
+ Task task = srcVertex.getTask(taskIndex);
+ if (task.getState() == TaskState.SUCCEEDED) {
+ successfulAttemptInteger = task.getSuccessfulAttempt().getID().getId();
+ vertexSuccessfulAttemptMap.put(taskIndex, successfulAttemptInteger);
+ }
+ }
+
+ if (successfulAttemptInteger == null) {
+ // Queue events and await a notification
+ pendingEvents.put(srcVertexName, tezEvent);
+ } else {
+ // Handle the event immediately.
+ if (taskAttemptIndex == successfulAttemptInteger) {
+ toForwardEvents.add((InputInitializerEvent) tezEvent.getEvent());
+ } // Otherwise the event can be dropped
+ }
+ }
+ sendEvents(toForwardEvents);
+ }
+
+ private void sendEvents(List<InputInitializerEvent> events) {
+ if (events != null && !events.isEmpty()) {
+ try {
+ initializer.handleInputInitializerEvent(events);
+ } catch (Exception e) {
+ throw new TezUncheckedException(
+ "Initializer for input: " + getInput().getName() + " on vertex: " + getVertexLogIdentifier() +
+ " failed to process events", e);
+ }
+ }
+ }
+
+ private void unregisterForTaskStatusUpdates() {
+ for (String vertexName : taskNotificationRegisteredVertices) {
+ stateChangeNotifier.unregisterForTaskSuccessUpdates(vertexName, this);
+ }
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
index 558fc61..dc18e9b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
@@ -33,6 +33,7 @@ import com.google.common.collect.SetMultimap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.records.TezTaskID;
import org.apache.tez.dag.records.TezVertexID;
/**
@@ -55,13 +56,11 @@ public class StateChangeNotifier {
this.lastKnowStatesMap = LinkedListMultimap.create();
}
+ // -------------- VERTEX STATE CHANGE SECTION ---------------
public void registerForVertexUpdates(String vertexName,
Set<org.apache.tez.dag.api.event.VertexState> stateSet,
VertexStateUpdateListener listener) {
- Preconditions.checkNotNull(vertexName, "VertexName cannot be null");
- Vertex vertex = dag.getVertex(vertexName);
- Preconditions.checkNotNull(vertex, "Vertex does not exist: " + vertexName);
- TezVertexID vertexId = vertex.getVertexId();
+ TezVertexID vertexId = validateAndGetVertexId(vertexName);
writeLock.lock();
// Read within the lock, to ensure a consistent view is seen.
List<VertexStateUpdate> previousUpdates = lastKnowStatesMap.get(vertexId);
@@ -88,14 +87,8 @@ public class StateChangeNotifier {
}
}
- // KKK Send out current state.
-
-
public void unregisterForVertexUpdates(String vertexName, VertexStateUpdateListener listener) {
- Preconditions.checkNotNull(vertexName, "VertexName cannot be null");
- Vertex vertex = dag.getVertex(vertexName);
- Preconditions.checkNotNull(vertex, "Vertex does not exist: " + vertexName);
- TezVertexID vertexId = vertex.getVertexId();
+ TezVertexID vertexId = validateAndGetVertexId(vertexName);
writeLock.lock();
try {
ListenerContainer listenerContainer = new ListenerContainer(listener, null);
@@ -125,6 +118,7 @@ public class StateChangeNotifier {
}
+
private static final class ListenerContainer {
final VertexStateUpdateListener listener;
final Set<org.apache.tez.dag.api.event.VertexState> states;
@@ -165,4 +159,64 @@ public class StateChangeNotifier {
return System.identityHashCode(listener);
}
}
+
+ // -------------- END OF VERTEX STATE CHANGE SECTION ---------------
+
+ // -------------- TASK STATE CHANGE SECTION ---------------
+
+ // Task updates are not buffered to avoid storing unnecessary information.
+ // Components (non user facing) which use this will receive notifications after registration.
+ // They will have to query task states, prior to registration.
+ // Currently only handling Task SUCCESS events.
+ private final SetMultimap<TezVertexID, TaskStateUpdateListener> taskListeners =
+ Multimaps.synchronizedSetMultimap(HashMultimap.<TezVertexID, TaskStateUpdateListener>create());
+ private final ReentrantReadWriteLock taskListenerLock = new ReentrantReadWriteLock();
+ private final ReentrantReadWriteLock.ReadLock taskReadLock = taskListenerLock.readLock();
+ private final ReentrantReadWriteLock.WriteLock taskWriteLock = taskListenerLock.writeLock();
+
+
+
+ public void registerForTaskSuccessUpdates(String vertexName, TaskStateUpdateListener listener) {
+ TezVertexID vertexId = validateAndGetVertexId(vertexName);
+ Preconditions.checkNotNull(listener, "listener cannot be null");
+ taskWriteLock.lock();
+ try {
+ taskListeners.put(vertexId, listener);
+ } finally {
+ taskWriteLock.unlock();
+ }
+ }
+
+ public void unregisterForTaskSuccessUpdates(String vertexName, TaskStateUpdateListener listener) {
+ TezVertexID vertexId = validateAndGetVertexId(vertexName);
+ Preconditions.checkNotNull(listener, "listener cannot be null");
+ taskWriteLock.lock();
+ try {
+ taskListeners.remove(vertexId, listener);
+ } finally {
+ taskWriteLock.unlock();
+ }
+ }
+
+ public void taskSucceeded(String vertexName, TezTaskID taskId, int attemptId) {
+ taskReadLock.lock();
+ try {
+ for (TaskStateUpdateListener listener : taskListeners.get(taskId.getVertexID())) {
+ listener.onTaskSucceeded(vertexName, taskId, attemptId);
+ }
+ } finally {
+ taskReadLock.unlock();
+ }
+ }
+
+ // -------------- END OF TASK STATE CHANGE SECTION ---------------
+
+
+ private TezVertexID validateAndGetVertexId(String vertexName) {
+ Preconditions.checkNotNull(vertexName, "VertexName cannot be null");
+ Vertex vertex = dag.getVertex(vertexName);
+ Preconditions.checkNotNull(vertex, "Vertex does not exist: " + vertexName);
+ return vertex.getVertexId();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.java
new file mode 100644
index 0000000..7c86991
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.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.dag.app.dag;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.api.oldrecords.TaskState;
+import org.apache.tez.dag.records.TezTaskID;
+
+@InterfaceAudience.Private
+/**
+ * This class should not be implemented by user facing APIs such as InputInitializer
+ */
+public interface TaskStateUpdateListener {
+
+ // Internal usage only. Currently only supporting onSuccess notifications for tasks.
+ // Exposing the taskID is ok, since this isn't public
+ public void onTaskSucceeded(String vertexName, TezTaskID taskId, int attemptId);
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 1dd711b..976f10f 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
@@ -29,6 +29,7 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import com.google.common.base.Preconditions;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -39,7 +40,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
import org.apache.hadoop.yarn.state.MultipleArcTransition;
import org.apache.hadoop.yarn.state.SingleArcTransition;
-import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.tez.common.counters.TezCounters;
@@ -53,6 +53,7 @@ import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerContext;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.StateChangeNotifier;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.dag.TaskAttempt;
import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
@@ -88,6 +89,8 @@ import org.apache.tez.runtime.api.OutputCommitter;
import org.apache.tez.runtime.api.impl.TezEvent;
import com.google.common.annotations.VisibleForTesting;
+import org.apache.tez.state.OnStateChangedCallback;
+import org.apache.tez.state.StateMachineTez;
/**
* Implementation of Task interface.
@@ -118,6 +121,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
private final ContainerContext containerContext;
@VisibleForTesting
long scheduledTime;
+ final StateChangeNotifier stateChangeNotifier;
private final List<TezEvent> tezEventsForTaskAttempts = new ArrayList<TezEvent>();
private static final List<TezEvent> EMPTY_TASK_ATTEMPT_TEZ_EVENTS =
@@ -138,6 +142,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
// Recovery related flags
boolean recoveryStartEventSeen = false;
+ private static final TaskStateChangedCallback STATE_CHANGED_CALLBACK = new TaskStateChangedCallback();
+
private static final StateMachineFactory
<TaskImpl, TaskStateInternal, TaskEventType, TaskEvent>
stateMachineFactory
@@ -261,7 +267,13 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
// create the topology tables
.installTopology();
- private final StateMachine<TaskStateInternal, TaskEventType, TaskEvent>
+ private void augmentStateMachine() {
+ stateMachine
+ .registerStateEnteredCallback(TaskStateInternal.SUCCEEDED,
+ STATE_CHANGED_CALLBACK);
+ }
+
+ private final StateMachineTez<TaskStateInternal, TaskEventType, TaskEvent, TaskImpl>
stateMachine;
// TODO: Recovery
@@ -318,7 +330,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
TaskAttemptListener taskAttemptListener,
Clock clock, TaskHeartbeatHandler thh, AppContext appContext,
boolean leafVertex, Resource resource,
- ContainerContext containerContext) {
+ ContainerContext containerContext,
+ StateChangeNotifier stateChangeNotifier) {
this.conf = conf;
this.clock = clock;
ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
@@ -333,11 +346,14 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
this.taskHeartbeatHandler = thh;
this.eventHandler = eventHandler;
this.appContext = appContext;
+ this.stateChangeNotifier = stateChangeNotifier;
this.leafVertex = leafVertex;
this.taskResource = resource;
this.containerContext = containerContext;
- stateMachine = stateMachineFactory.make(this);
+ stateMachine = new StateMachineTez<TaskStateInternal, TaskEventType, TaskEvent, TaskImpl>(
+ stateMachineFactory.make(this), this);
+ augmentStateMachine();
}
@Override
@@ -1423,6 +1439,27 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
}
}
+ private static class TaskStateChangedCallback
+ implements OnStateChangedCallback<TaskStateInternal, TaskImpl> {
+
+ @Override
+ public void onStateChanged(TaskImpl task, TaskStateInternal taskStateInternal) {
+ // Only registered for SUCCEEDED notifications at the moment
+ Preconditions.checkState(taskStateInternal == TaskStateInternal.SUCCEEDED);
+ TaskAttempt successfulAttempt = task.getSuccessfulAttempt();
+ // TODO TEZ-1577.
+ // This is a horrible hack to get around recovery issues. Without this, recovery would fail
+ // for successful vertices.
+ // With this, recovery will end up failing for DAGs making use of InputInitializerEvents
+ int succesfulAttemptInt = -1;
+ if (successfulAttempt != null) {
+ succesfulAttemptInt = successfulAttempt.getID().getId();
+ }
+ task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskId(),
+ succesfulAttemptInt);
+ }
+ }
+
private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg) {
if (commitAttempt != null && commitAttempt.equals(attempt)) {
LOG.info("Removing commit attempt: " + commitAttempt);
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 6437e5b..594c651 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
@@ -127,7 +127,7 @@ import org.apache.tez.dag.app.dag.impl.DAGImpl.VertexGroupInfo;
import org.apache.tez.dag.history.DAGHistoryEvent;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
import org.apache.tez.dag.history.events.VertexFinishedEvent;
import org.apache.tez.dag.history.events.VertexInitializedEvent;
import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
@@ -244,6 +244,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
private boolean vertexAlreadyInitialized = false;
+ @VisibleForTesting
+ final List<TezEvent> pendingInitializerEvents = new LinkedList<TezEvent>();
+
protected static final
StateMachineFactory<VertexImpl, VertexState, VertexEventType, VertexEvent>
stateMachineFactory
@@ -261,6 +264,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
EnumSet.of(VertexState.NEW),
VertexEventType.V_NULL_EDGE_INITIALIZED,
new NullEdgeInitializedTransition())
+ .addTransition(VertexState.NEW, VertexState.NEW,
+ VertexEventType.V_ROUTE_EVENT,
+ ROUTE_EVENT_TRANSITION)
+ .addTransition(VertexState.NEW, VertexState.NEW,
+ VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED,
+ SOURCE_TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION)
.addTransition
(VertexState.NEW,
EnumSet.of(VertexState.NEW, VertexState.INITED,
@@ -1079,8 +1088,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
return recoveredState;
case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
- VertexDataMovementEventsGeneratedEvent vEvent =
- (VertexDataMovementEventsGeneratedEvent) historyEvent;
+ VertexRecoverableEventsGeneratedEvent vEvent =
+ (VertexRecoverableEventsGeneratedEvent) historyEvent;
this.recoveredEvents.addAll(vEvent.getTezEvents());
return recoveredState;
default:
@@ -1774,7 +1783,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
(this.targetVertices != null ?
this.targetVertices.isEmpty() : true),
this.taskResource,
- conContext);
+ conContext,
+ this.stateChangeNotifier);
this.addTask(task);
if(LOG.isDebugEnabled()) {
LOG.debug("Created task for vertex " + logIdentifier + ": " +
@@ -2218,6 +2228,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
this.getVertexId(), Collections.singletonList(tezEvent), true));
}
continue;
+ } else if (tezEvent.getEventType() == EventType.ROOT_INPUT_INITIALIZER_EVENT) {
+ // The event has the relevant target information
+ InputInitializerEvent iiEvent = (InputInitializerEvent) tezEvent.getEvent();
+ iiEvent.setSourceVertexName(vertexName);
+ eventHandler.handle(new VertexEventRouteEvent(
+ getDAG().getVertex(iiEvent.getTargetVertexName()).getVertexId(),
+ Collections.singletonList(tezEvent), true));
+ continue;
}
Vertex destVertex = getDAG().getVertex(sourceMeta.getEdgeVertexName());
@@ -2661,6 +2679,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
+ vertex.logIdentifier + ". Starting root input initializers: "
+ vertex.inputsWithInitializers.size());
vertex.rootInputInitializerManager.runInputInitializers(inputList);
+ // Send pending rootInputInitializerEvents
+ vertex.rootInputInitializerManager.handleInitializerEvents(vertex.pendingInitializerEvents);
+ vertex.pendingInitializerEvents.clear();
return VertexState.INITIALIZING;
} else {
boolean hasOneToOneUninitedSource = false;
@@ -2706,6 +2727,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
// state. This is handled in RootInputInitializedTransition specially.
vertex.initWaitsForRootInitializers = true;
vertex.rootInputInitializerManager.runInputInitializers(inputList);
+ // Send pending rootInputInitializerEvents
+ vertex.rootInputInitializerManager.handleInitializerEvents(vertex.pendingInitializerEvents);
+ vertex.pendingInitializerEvents.clear();
return VertexState.INITIALIZING;
}
if (!vertex.uninitializedEdges.isEmpty()) {
@@ -2795,6 +2819,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
if (vertex.numInitializedInputs == vertex.inputsWithInitializers.size()) {
// All inputs initialized, shutdown the initializer.
vertex.rootInputInitializerManager.shutdown();
+ vertex.rootInputInitializerManager = null;
}
// done. check if we need to do the initialization
@@ -3064,6 +3089,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
if (vertex.rootInputInitializerManager != null) {
vertex.rootInputInitializerManager.shutdown();
+ vertex.rootInputInitializerManager = null;
}
vertex.finished(VertexState.FAILED,
VertexTerminationCause.ROOT_INPUT_INIT_FAILURE);
@@ -3102,6 +3128,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
super.transition(vertex, event);
if (vertex.rootInputInitializerManager != null) {
vertex.rootInputInitializerManager.shutdown();
+ vertex.rootInputInitializerManager = null;
}
}
}
@@ -3146,17 +3173,19 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
+ " with state: " + completionEvent.getTaskAttemptState()
+ " vertexState: " + vertex.getState());
+
if (TaskAttemptStateInternal.SUCCEEDED.equals(completionEvent
.getTaskAttemptState())) {
vertex.numSuccessSourceAttemptCompletions++;
+
if (vertex.getState() == VertexState.RUNNING) {
+ // Inform the vertex manager about the source task completing.
vertex.vertexManager.onSourceTaskCompleted(completionEvent
.getTaskAttemptId().getTaskID());
} else {
vertex.pendingReportedSrcCompletions.add(completionEvent.getTaskAttemptId());
}
}
-
}
}
@@ -3349,7 +3378,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
if (vertex.getAppContext().isRecoveryEnabled()
&& !recovered
&& !tezEvents.isEmpty()) {
- List<TezEvent> dataMovementEvents =
+ List<TezEvent> recoveryEvents =
Lists.newArrayList();
for (TezEvent tezEvent : tezEvents) {
if (!isEventFromVertex(vertex, tezEvent.getSourceInfo())) {
@@ -3357,14 +3386,15 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
if (tezEvent.getEventType().equals(EventType.COMPOSITE_DATA_MOVEMENT_EVENT)
|| tezEvent.getEventType().equals(EventType.DATA_MOVEMENT_EVENT)
- || tezEvent.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
- dataMovementEvents.add(tezEvent);
+ || tezEvent.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)
+ || tezEvent.getEventType().equals(EventType.ROOT_INPUT_INITIALIZER_EVENT)) {
+ recoveryEvents.add(tezEvent);
}
}
- if (!dataMovementEvents.isEmpty()) {
- VertexDataMovementEventsGeneratedEvent historyEvent =
- new VertexDataMovementEventsGeneratedEvent(vertex.vertexId,
- dataMovementEvents);
+ if (!recoveryEvents.isEmpty()) {
+ VertexRecoverableEventsGeneratedEvent historyEvent =
+ new VertexRecoverableEventsGeneratedEvent(vertex.vertexId,
+ recoveryEvents);
vertex.appContext.getHistoryHandler().handle(
new DAGHistoryEvent(vertex.getDAGId(), historyEvent));
}
@@ -3431,6 +3461,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
break;
case VERTEX_MANAGER_EVENT:
{
+ // VM events on task success only can be changed as part of TEZ-1532
VertexManagerEvent vmEvent = (VertexManagerEvent) tezEvent.getEvent();
Vertex target = vertex.getDAG().getVertex(vmEvent.getTargetVertexName());
Preconditions.checkArgument(target != null,
@@ -3449,9 +3480,25 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
InputInitializerEvent riEvent = (InputInitializerEvent) tezEvent.getEvent();
Vertex target = vertex.getDAG().getVertex(riEvent.getTargetVertexName());
Preconditions.checkArgument(target != null,
- "Event sent to unkown vertex: " + riEvent.getTargetVertexName());
+ "Event sent to unknown vertex: " + riEvent.getTargetVertexName());
+ riEvent.setSourceVertexName(tezEvent.getSourceInfo().getTaskVertexName());
if (target == vertex) {
- vertex.rootInputInitializerManager.handleInitializerEvent(riEvent);
+ if (vertex.rootInputDescriptors == null ||
+ !vertex.rootInputDescriptors.containsKey(riEvent.getTargetInputName())) {
+ throw new TezUncheckedException(
+ "InputInitializerEvent targeted at unknown initializer on vertex " +
+ vertex.logIdentifier + ", Event=" + riEvent);
+ }
+ if (vertex.getState() == VertexState.NEW) {
+ vertex.pendingInitializerEvents.add(tezEvent);
+ } else if (vertex.getState() == VertexState.INITIALIZING) {
+ vertex.rootInputInitializerManager.handleInitializerEvents(Collections.singletonList(tezEvent));
+ } else {
+ // Currently, INITED and subsequent states means Initializer complete / failure
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Dropping event" + tezEvent + " since state is not INITIALIZING in " + vertex.getLogIdentifier() + ", state=" + vertex.getState());
+ }
+ }
} else {
checkEventSourceMetadata(vertex, sourceMeta);
vertex.eventHandler.handle(new VertexEventRouteEvent(target.getVertexId(),
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
deleted file mode 100644
index 7ae73be..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.history.events;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.EnumSet;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.common.ProtoConverters;
-import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.HistoryEventType;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.dag.recovery.records.RecoveryProtos;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.TezDataMovementEventProto;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexDataMovementEventsGeneratedProto;
-import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
-import org.apache.tez.runtime.api.events.DataMovementEvent;
-import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import org.apache.tez.runtime.api.impl.EventMetaData;
-import org.apache.tez.runtime.api.impl.EventType;
-import org.apache.tez.runtime.api.impl.TezEvent;
-
-import com.google.common.collect.Lists;
-
-public class VertexDataMovementEventsGeneratedEvent implements HistoryEvent {
-
- private static final Log LOG = LogFactory.getLog(
- VertexDataMovementEventsGeneratedEvent.class);
- private List<TezEvent> events;
- private TezVertexID vertexID;
-
- public VertexDataMovementEventsGeneratedEvent(TezVertexID vertexID,
- List<TezEvent> events) {
- this.vertexID = vertexID;
- this.events = Lists.newArrayListWithCapacity(events.size());
- for (TezEvent event : events) {
- if (EnumSet.of(EventType.DATA_MOVEMENT_EVENT,
- EventType.COMPOSITE_DATA_MOVEMENT_EVENT,
- EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)
- .contains(event.getEventType())) {
- this.events.add(event);
- }
- }
- if (events.isEmpty()) {
- throw new RuntimeException("Invalid creation of VertexDataMovementEventsGeneratedEvent"
- + ", no data movement/information events provided");
- }
- }
-
- public VertexDataMovementEventsGeneratedEvent() {
- }
-
- @Override
- public HistoryEventType getEventType() {
- return HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED;
- }
-
- @Override
- public boolean isRecoveryEvent() {
- return true;
- }
-
- @Override
- public boolean isHistoryEvent() {
- return false;
- }
-
- static RecoveryProtos.EventMetaDataProto convertEventMetaDataToProto(
- EventMetaData eventMetaData) {
- RecoveryProtos.EventMetaDataProto.Builder builder =
- RecoveryProtos.EventMetaDataProto.newBuilder()
- .setProducerConsumerType(eventMetaData.getEventGenerator().ordinal())
- .setEdgeVertexName(eventMetaData.getEdgeVertexName())
- .setTaskVertexName(eventMetaData.getTaskVertexName());
- if (eventMetaData.getTaskAttemptID() != null) {
- builder.setTaskAttemptId(eventMetaData.getTaskAttemptID().toString());
- }
- return builder.build();
- }
-
- static EventMetaData convertEventMetaDataFromProto(
- RecoveryProtos.EventMetaDataProto proto) {
- TezTaskAttemptID attemptID = null;
- if (proto.hasTaskAttemptId()) {
- attemptID = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
- }
- return new EventMetaData(
- EventMetaData.EventProducerConsumerType.values()[proto.getProducerConsumerType()],
- proto.getTaskVertexName(),
- proto.getEdgeVertexName(),
- attemptID);
- }
-
- public VertexDataMovementEventsGeneratedProto toProto() {
- List<TezDataMovementEventProto> tezEventProtos = null;
- if (events != null) {
- tezEventProtos = Lists.newArrayListWithCapacity(events.size());
- for (TezEvent event : events) {
- TezDataMovementEventProto.Builder evtBuilder =
- TezDataMovementEventProto.newBuilder();
- if (event.getEventType().equals(EventType.COMPOSITE_DATA_MOVEMENT_EVENT)) {
- evtBuilder.setCompositeDataMovementEvent(
- ProtoConverters.convertCompositeDataMovementEventToProto(
- (CompositeDataMovementEvent) event.getEvent()));
- } else if (event.getEventType().equals(EventType.DATA_MOVEMENT_EVENT)) {
- evtBuilder.setDataMovementEvent(
- ProtoConverters.convertDataMovementEventToProto(
- (DataMovementEvent) event.getEvent()));
- } else if (event.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
- evtBuilder.setRootInputDataInformationEvent(
- ProtoConverters.convertRootInputDataInformationEventToProto(
- (InputDataInformationEvent) event.getEvent()));
- }
- if (event.getSourceInfo() != null) {
- evtBuilder.setSourceInfo(convertEventMetaDataToProto(event.getSourceInfo()));
- }
- if (event.getDestinationInfo() != null) {
- evtBuilder.setDestinationInfo(convertEventMetaDataToProto(event.getDestinationInfo()));
- }
- tezEventProtos.add(evtBuilder.build());
- }
- }
- return VertexDataMovementEventsGeneratedProto.newBuilder()
- .setVertexId(vertexID.toString())
- .addAllTezDataMovementEvent(tezEventProtos)
- .build();
- }
-
- public void fromProto(VertexDataMovementEventsGeneratedProto proto) {
- this.vertexID = TezVertexID.fromString(proto.getVertexId());
- int eventCount = proto.getTezDataMovementEventCount();
- if (eventCount > 0) {
- this.events = Lists.newArrayListWithCapacity(eventCount);
- }
- for (TezDataMovementEventProto eventProto :
- proto.getTezDataMovementEventList()) {
- Event evt = null;
- if (eventProto.hasCompositeDataMovementEvent()) {
- evt = ProtoConverters.convertCompositeDataMovementEventFromProto(
- eventProto.getCompositeDataMovementEvent());
- } else if (eventProto.hasDataMovementEvent()) {
- evt = ProtoConverters.convertDataMovementEventFromProto(
- eventProto.getDataMovementEvent());
- } else if (eventProto.hasRootInputDataInformationEvent()) {
- evt = ProtoConverters.convertRootInputDataInformationEventFromProto(
- eventProto.getRootInputDataInformationEvent());
- }
- EventMetaData sourceInfo = null;
- EventMetaData destinationInfo = null;
- if (eventProto.hasSourceInfo()) {
- sourceInfo = convertEventMetaDataFromProto(eventProto.getSourceInfo());
- }
- if (eventProto.hasDestinationInfo()) {
- destinationInfo = convertEventMetaDataFromProto(eventProto.getDestinationInfo());
- }
- TezEvent tezEvent = new TezEvent(evt, sourceInfo);
- tezEvent.setDestinationInfo(destinationInfo);
- this.events.add(tezEvent);
- }
- }
-
- @Override
- public void toProtoStream(OutputStream outputStream) throws IOException {
- toProto().writeDelimitedTo(outputStream);
- }
-
- @Override
- public void fromProtoStream(InputStream inputStream) throws IOException {
- VertexDataMovementEventsGeneratedProto proto =
- VertexDataMovementEventsGeneratedProto.parseDelimitedFrom(inputStream);
- if (proto == null) {
- throw new IOException("No data found in stream");
- }
- fromProto(proto);
- }
-
- @Override
- public String toString() {
- return "vertexId=" + vertexID.toString()
- + ", eventCount=" + (events != null ? events.size() : "null");
-
- }
-
- public TezVertexID getVertexID() {
- return this.vertexID;
- }
-
- public List<TezEvent> getTezEvents() {
- return this.events;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java
new file mode 100644
index 0000000..a9f1fd2
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.history.events;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.common.ProtoConverters;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TezDataMovementEventProto;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexDataMovementEventsGeneratedProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventType;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+import com.google.common.collect.Lists;
+
+// TODO PreCommit - rename this to VertexRecoverableEventGeneratedEvent
+public class VertexRecoverableEventsGeneratedEvent implements HistoryEvent {
+
+ private static final Log LOG = LogFactory.getLog(
+ VertexRecoverableEventsGeneratedEvent.class);
+ private List<TezEvent> events;
+ private TezVertexID vertexID;
+
+ public VertexRecoverableEventsGeneratedEvent(TezVertexID vertexID,
+ List<TezEvent> events) {
+ this.vertexID = vertexID;
+ this.events = Lists.newArrayListWithCapacity(events.size());
+ for (TezEvent event : events) {
+ if (EnumSet.of(EventType.DATA_MOVEMENT_EVENT,
+ EventType.COMPOSITE_DATA_MOVEMENT_EVENT,
+ EventType.ROOT_INPUT_DATA_INFORMATION_EVENT,
+ EventType.ROOT_INPUT_INITIALIZER_EVENT)
+ .contains(event.getEventType())) {
+ this.events.add(event);
+ }
+ }
+ if (events.isEmpty()) {
+ throw new RuntimeException("Invalid creation of VertexDataMovementEventsGeneratedEvent"
+ + ", no data movement/information events provided");
+ }
+ }
+
+ public VertexRecoverableEventsGeneratedEvent() {
+ }
+
+ @Override
+ public HistoryEventType getEventType() {
+ return HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED;
+ }
+
+ @Override
+ public boolean isRecoveryEvent() {
+ return true;
+ }
+
+ @Override
+ public boolean isHistoryEvent() {
+ return false;
+ }
+
+ static RecoveryProtos.EventMetaDataProto convertEventMetaDataToProto(
+ EventMetaData eventMetaData) {
+ RecoveryProtos.EventMetaDataProto.Builder builder =
+ RecoveryProtos.EventMetaDataProto.newBuilder()
+ .setProducerConsumerType(eventMetaData.getEventGenerator().ordinal())
+ .setEdgeVertexName(eventMetaData.getEdgeVertexName())
+ .setTaskVertexName(eventMetaData.getTaskVertexName());
+ if (eventMetaData.getTaskAttemptID() != null) {
+ builder.setTaskAttemptId(eventMetaData.getTaskAttemptID().toString());
+ }
+ return builder.build();
+ }
+
+ static EventMetaData convertEventMetaDataFromProto(
+ RecoveryProtos.EventMetaDataProto proto) {
+ TezTaskAttemptID attemptID = null;
+ if (proto.hasTaskAttemptId()) {
+ attemptID = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
+ }
+ return new EventMetaData(
+ EventMetaData.EventProducerConsumerType.values()[proto.getProducerConsumerType()],
+ proto.getTaskVertexName(),
+ proto.getEdgeVertexName(),
+ attemptID);
+ }
+
+ public VertexDataMovementEventsGeneratedProto toProto() {
+ List<TezDataMovementEventProto> tezEventProtos = null;
+ if (events != null) {
+ tezEventProtos = Lists.newArrayListWithCapacity(events.size());
+ for (TezEvent event : events) {
+ TezDataMovementEventProto.Builder evtBuilder =
+ TezDataMovementEventProto.newBuilder();
+ if (event.getEventType().equals(EventType.COMPOSITE_DATA_MOVEMENT_EVENT)) {
+ evtBuilder.setCompositeDataMovementEvent(
+ ProtoConverters.convertCompositeDataMovementEventToProto(
+ (CompositeDataMovementEvent) event.getEvent()));
+ } else if (event.getEventType().equals(EventType.DATA_MOVEMENT_EVENT)) {
+ evtBuilder.setDataMovementEvent(
+ ProtoConverters.convertDataMovementEventToProto(
+ (DataMovementEvent) event.getEvent()));
+ } else if (event.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
+ evtBuilder.setRootInputDataInformationEvent(
+ ProtoConverters.convertRootInputDataInformationEventToProto(
+ (InputDataInformationEvent) event.getEvent()));
+ } else if (event.getEventType().equals(EventType.ROOT_INPUT_INITIALIZER_EVENT)) {
+ evtBuilder.setInputInitializerEvent(ProtoConverters
+ .convertRootInputInitializerEventToProto((InputInitializerEvent) event.getEvent()));
+ }
+ if (event.getSourceInfo() != null) {
+ evtBuilder.setSourceInfo(convertEventMetaDataToProto(event.getSourceInfo()));
+ }
+ if (event.getDestinationInfo() != null) {
+ evtBuilder.setDestinationInfo(convertEventMetaDataToProto(event.getDestinationInfo()));
+ }
+ tezEventProtos.add(evtBuilder.build());
+ }
+ }
+ return VertexDataMovementEventsGeneratedProto.newBuilder()
+ .setVertexId(vertexID.toString())
+ .addAllTezDataMovementEvent(tezEventProtos)
+ .build();
+ }
+
+ public void fromProto(VertexDataMovementEventsGeneratedProto proto) {
+ this.vertexID = TezVertexID.fromString(proto.getVertexId());
+ int eventCount = proto.getTezDataMovementEventCount();
+ if (eventCount > 0) {
+ this.events = Lists.newArrayListWithCapacity(eventCount);
+ }
+ for (TezDataMovementEventProto eventProto :
+ proto.getTezDataMovementEventList()) {
+ Event evt = null;
+ if (eventProto.hasCompositeDataMovementEvent()) {
+ evt = ProtoConverters.convertCompositeDataMovementEventFromProto(
+ eventProto.getCompositeDataMovementEvent());
+ } else if (eventProto.hasDataMovementEvent()) {
+ evt = ProtoConverters.convertDataMovementEventFromProto(
+ eventProto.getDataMovementEvent());
+ } else if (eventProto.hasRootInputDataInformationEvent()) {
+ evt = ProtoConverters.convertRootInputDataInformationEventFromProto(
+ eventProto.getRootInputDataInformationEvent());
+ } else if (eventProto.hasInputInitializerEvent()) {
+ evt = ProtoConverters.convertRootInputInitializerEventFromProto(
+ eventProto.getInputInitializerEvent());
+ }
+ EventMetaData sourceInfo = null;
+ EventMetaData destinationInfo = null;
+ if (eventProto.hasSourceInfo()) {
+ sourceInfo = convertEventMetaDataFromProto(eventProto.getSourceInfo());
+ }
+ if (eventProto.hasDestinationInfo()) {
+ destinationInfo = convertEventMetaDataFromProto(eventProto.getDestinationInfo());
+ }
+ TezEvent tezEvent = new TezEvent(evt, sourceInfo);
+ tezEvent.setDestinationInfo(destinationInfo);
+ this.events.add(tezEvent);
+ }
+ }
+
+ @Override
+ public void toProtoStream(OutputStream outputStream) throws IOException {
+ toProto().writeDelimitedTo(outputStream);
+ }
+
+ @Override
+ public void fromProtoStream(InputStream inputStream) throws IOException {
+ VertexDataMovementEventsGeneratedProto proto =
+ VertexDataMovementEventsGeneratedProto.parseDelimitedFrom(inputStream);
+ if (proto == null) {
+ throw new IOException("No data found in stream");
+ }
+ fromProto(proto);
+ }
+
+ @Override
+ public String toString() {
+ return "vertexId=" + vertexID.toString()
+ + ", eventCount=" + (events != null ? events.size() : "null");
+
+ }
+
+ public TezVertexID getVertexID() {
+ return this.vertexID;
+ }
+
+ public List<TezEvent> getTezEvents() {
+ return this.events;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/proto/HistoryEvents.proto
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/proto/HistoryEvents.proto b/tez-dag/src/main/proto/HistoryEvents.proto
index 821612a..93f217f 100644
--- a/tez-dag/src/main/proto/HistoryEvents.proto
+++ b/tez-dag/src/main/proto/HistoryEvents.proto
@@ -180,6 +180,7 @@ message TezDataMovementEventProto {
optional DataMovementEventProto data_movement_event = 3;
optional CompositeEventProto composite_data_movement_event = 4;
optional RootInputDataInformationEventProto root_input_data_information_event = 5;
+ optional RootInputInitializerEventProto input_initializer_event = 6;
}
message VertexDataMovementEventsGeneratedProto {
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
index f53643f..63cd9c5 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
@@ -25,6 +25,9 @@ import static org.junit.Assert.fail;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
import java.util.ArrayList;
import java.util.HashMap;
@@ -53,6 +56,7 @@ import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerContext;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.StateChangeNotifier;
import org.apache.tez.dag.app.dag.TaskStateInternal;
import org.apache.tez.dag.app.dag.TaskTerminationCause;
import org.apache.tez.dag.app.dag.Vertex;
@@ -494,6 +498,8 @@ public class TestTaskImpl {
// The task should now have succeeded
assertTaskSucceededState();
+ verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId),
+ eq(mockTask.getLastAttempt().getID().getId()));
eventHandler.events.clear();
// Now fail the attempt after it has succeeded
@@ -548,7 +554,7 @@ public class TestTaskImpl {
private List<MockTaskAttemptImpl> taskAttempts = new LinkedList<MockTaskAttemptImpl>();
private Vertex vertex;
TaskLocationHint locationHint;
-
+
public MockTaskImpl(TezVertexID vertexId, int partition,
EventHandler eventHandler, Configuration conf,
TaskAttemptListener taskAttemptListener, Clock clock,
@@ -557,7 +563,7 @@ public class TestTaskImpl {
ContainerContext containerContext, Vertex vertex) {
super(vertexId, partition, eventHandler, conf, taskAttemptListener,
clock, thh, appContext, leafVertex, resource,
- containerContext);
+ containerContext, mock(StateChangeNotifier.class));
this.vertex = vertex;
this.locationHint = locationHint;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
index c5153b6..bd13ffe 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
@@ -47,6 +47,7 @@ import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerContext;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.StateChangeNotifier;
import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
import org.apache.tez.dag.app.dag.TaskStateInternal;
import org.apache.tez.dag.app.dag.Vertex;
@@ -186,7 +187,7 @@ public class TestTaskRecovery {
new Configuration(), mock(TaskAttemptListener.class),
new SystemClock(), mock(TaskHeartbeatHandler.class),
mockAppContext, false, Resource.newInstance(1, 1),
- mock(ContainerContext.class));
+ mock(ContainerContext.class), mock(StateChangeNotifier.class));
Map<String, OutputCommitter> committers =
new HashMap<String, OutputCommitter>();
[23/25] git commit: TEZ-1533. Request Events more often if a complete
set of events is received by a task. (sseth)
Posted by je...@apache.org.
TEZ-1533. Request Events more often if a complete set of events is
received by a task. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/8bd2ca35
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/8bd2ca35
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/8bd2ca35
Branch: refs/heads/TEZ-8
Commit: 8bd2ca352664fab8d248593f60faa6c0ff223920
Parents: 3b34c41
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Sep 16 03:54:45 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Sep 16 03:54:45 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/tez/runtime/task/TaskReporter.java | 53 ++++++---
.../tez/runtime/task/TestTaskReporter.java | 115 +++++++++++++++++++
3 files changed, 151 insertions(+), 18 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/8bd2ca35/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ca4e7e2..83048bb 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -42,6 +42,7 @@ ALL CHANGES
of DAG submission
TEZ-1571. Add create method for DataSinkDescriptor.
TEZ-1585. Memory leak in tez session mode.
+ TEZ-1533. Request Events more often if a complete set of events is received by a task.
Release 0.5.0: 2014-09-03
http://git-wip-us.apache.org/repos/asf/tez/blob/8bd2ca35/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java b/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
index fcb8778..15dcbb0 100644
--- a/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
@@ -115,7 +115,8 @@ public class TaskReporter {
heartbeatExecutor.shutdownNow();
}
- private static class HeartbeatCallable implements Callable<Boolean> {
+ @VisibleForTesting
+ static class HeartbeatCallable implements Callable<Boolean> {
private static final int LOG_COUNTER_START_INTERVAL = 5000; // 5 seconds
private static final float LOG_COUNTER_BACKOFF = 1.3f;
@@ -172,25 +173,30 @@ public class TaskReporter {
public Boolean call() throws Exception {
// Heartbeat only for active tasks. Errors, etc will be reported directly.
while (!task.isTaskDone() && !task.hadFatalError()) {
- boolean result = heartbeat(null);
- if (!result) {
+ ResponseWrapper response = heartbeat(null);
+
+ if (response.shouldDie) {
// AM sent a shouldDie=true
LOG.info("Asked to die via task heartbeat");
return false;
- }
- lock.lock();
- try {
- boolean interrupted = condition.await(pollInterval, TimeUnit.MILLISECONDS);
- if (!interrupted) {
- nonOobHeartbeatCounter++;
+ } else {
+ if (response.numEvents < maxEventsToGet) {
+ // Wait before sending another heartbeat. Otherwise consider as an OOB heartbeat
+ lock.lock();
+ try {
+ boolean interrupted = condition.await(pollInterval, TimeUnit.MILLISECONDS);
+ if (!interrupted) {
+ nonOobHeartbeatCounter++;
+ }
+ } finally {
+ lock.unlock();
+ }
}
- } finally {
- lock.unlock();
}
}
int pendingEventCount = eventsToSend.size();
if (pendingEventCount > 0) {
- LOG.warn("Exiting TaskReporter therad with pending queue size=" + pendingEventCount);
+ LOG.warn("Exiting TaskReporter thread with pending queue size=" + pendingEventCount);
}
return true;
}
@@ -203,7 +209,7 @@ public class TaskReporter {
* @throws TezException
* indicates an exception somewhere in the AM.
*/
- private synchronized boolean heartbeat(Collection<TezEvent> eventsArg) throws IOException,
+ private synchronized ResponseWrapper heartbeat(Collection<TezEvent> eventsArg) throws IOException,
TezException {
if (eventsArg != null) {
@@ -247,7 +253,7 @@ public class TaskReporter {
if (response.shouldDie()) {
LOG.info("Received should die response from AM");
- return false;
+ return new ResponseWrapper(true, 1);
}
if (response.getLastRequestId() != requestId) {
throw new TezException("AM and Task out of sync" + ", responseReqId="
@@ -256,6 +262,7 @@ public class TaskReporter {
// The same umbilical is used by multiple tasks. Problematic in the case where multiple tasks
// are running using the same umbilical.
+ int numEventsReceived = 0;
if (task.isTaskDone() || task.hadFatalError()) {
if (response.getEvents() != null && !response.getEvents().isEmpty()) {
LOG.warn("Current task already complete, Ignoring all event in"
@@ -268,11 +275,11 @@ public class TaskReporter {
+ task.getTaskAttemptID() + ", eventCount=" + response.getEvents().size());
}
// This should ideally happen in a separate thread
+ numEventsReceived = response.getEvents().size();
task.handleEvents(response.getEvents());
}
}
- return true;
-
+ return new ResponseWrapper(false, numEventsReceived);
}
public void markComplete() {
@@ -308,7 +315,7 @@ public class TaskReporter {
task.getProgress()), updateEventMetadata);
TezEvent taskCompletedEvent = new TezEvent(new TaskAttemptCompletedEvent(),
updateEventMetadata);
- return heartbeat(Lists.newArrayList(statusUpdateEvent, taskCompletedEvent));
+ return !heartbeat(Lists.newArrayList(statusUpdateEvent, taskCompletedEvent)).shouldDie;
}
/**
@@ -334,7 +341,7 @@ public class TaskReporter {
}
TezEvent taskAttemptFailedEvent = new TezEvent(new TaskAttemptFailedEvent(diagnostics),
srcMeta == null ? updateEventMetadata : srcMeta);
- return heartbeat(Lists.newArrayList(statusUpdateEvent, taskAttemptFailedEvent));
+ return !heartbeat(Lists.newArrayList(statusUpdateEvent, taskAttemptFailedEvent)).shouldDie;
}
private void addEvents(TezTaskAttemptID taskAttemptID, Collection<TezEvent> events) {
@@ -381,4 +388,14 @@ public class TaskReporter {
public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException {
return umbilical.canCommit(taskAttemptID);
}
+
+ private static final class ResponseWrapper {
+ boolean shouldDie;
+ int numEvents;
+
+ private ResponseWrapper(boolean shouldDie, int numEvents) {
+ this.shouldDie = shouldDie;
+ this.numEvents = numEvents;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8bd2ca35/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java b/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java
new file mode 100644
index 0000000..de03307
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.task;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.collect.Lists;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+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.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestTaskReporter {
+
+ @Test(timeout = 10000)
+ public void testContinuousHeartbeatsOnMaxEvents() throws Exception {
+
+ final Object lock = new Object();
+ final AtomicBoolean hb2Done = new AtomicBoolean(false);
+
+ TezTaskUmbilicalProtocol mockUmbilical = mock(TezTaskUmbilicalProtocol.class);
+ doAnswer(new Answer() {
+ @Override
+ public Object answer(InvocationOnMock invocation) throws Throwable {
+ Object[] args = invocation.getArguments();
+ TezHeartbeatRequest request = (TezHeartbeatRequest) args[0];
+ if (request.getRequestId() == 1 || request.getRequestId() == 2) {
+ TezHeartbeatResponse response = new TezHeartbeatResponse(createEvents(5));
+ response.setLastRequestId(request.getRequestId());
+ return response;
+ } else if (request.getRequestId() == 3) {
+ TezHeartbeatResponse response = new TezHeartbeatResponse(createEvents(1));
+ response.setLastRequestId(request.getRequestId());
+ synchronized (lock) {
+ hb2Done.set(true);
+ lock.notify();
+ }
+ return response;
+ } else {
+ throw new TezUncheckedException("Invalid request id for test: " + request.getRequestId());
+ }
+ }
+ }).when(mockUmbilical).heartbeat(any(TezHeartbeatRequest.class));
+
+ TezTaskAttemptID mockTaskAttemptId = mock(TezTaskAttemptID.class);
+ LogicalIOProcessorRuntimeTask mockTask = mock(LogicalIOProcessorRuntimeTask.class);
+ doReturn("vertexName").when(mockTask).getVertexName();
+ doReturn(mockTaskAttemptId).when(mockTask).getTaskAttemptID();
+
+ // Setup the sleep time to be way higher than the test timeout
+ TaskReporter.HeartbeatCallable heartbeatCallable =
+ new TaskReporter.HeartbeatCallable(mockTask, mockUmbilical, 100000, 100000, 5,
+ new AtomicLong(0),
+ "containerIdStr");
+
+ ExecutorService executor = Executors.newSingleThreadExecutor();
+ executor.submit(heartbeatCallable);
+ try {
+ synchronized (lock) {
+ if (!hb2Done.get()) {
+ lock.wait();
+ }
+ }
+ verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class));
+ Thread.sleep(2000l);
+ // Sleep for 2 seconds, less than the callable sleep time. No more invocations.
+ verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class));
+ } finally {
+ executor.shutdownNow();
+ }
+
+ }
+
+ private List<TezEvent> createEvents(int numEvents) {
+ List<TezEvent> list = Lists.newArrayListWithCapacity(numEvents);
+ for (int i = 0; i < numEvents; i++) {
+ list.add(mock(TezEvent.class));
+ }
+ return list;
+ }
+}
[14/25] git commit: TEZ-1578. Remove TeraSort from Tez codebase.
(hitesh)
Posted by je...@apache.org.
TEZ-1578. Remove TeraSort from Tez codebase. (hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/8e382b34
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/8e382b34
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/8e382b34
Branch: refs/heads/TEZ-8
Commit: 8e382b34d9487faea56ac0a8a31c53745aacdc3d
Parents: b62298e
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 11:11:47 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 11:11:47 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 3 +
.../tez/mapreduce/examples/ExampleDriver.java | 9 -
.../mapreduce/examples/terasort/GenSort.java | 251 -------------
.../mapreduce/examples/terasort/Random16.java | 374 -------------------
.../examples/terasort/TeraChecksum.java | 103 -----
.../mapreduce/examples/terasort/TeraGen.java | 311 ---------------
.../examples/terasort/TeraInputFormat.java | 353 -----------------
.../examples/terasort/TeraOutputFormat.java | 112 ------
.../examples/terasort/TeraScheduler.java | 253 -------------
.../mapreduce/examples/terasort/TeraSort.java | 335 -----------------
.../examples/terasort/TeraValidate.java | 188 ----------
.../mapreduce/examples/terasort/Unsigned16.java | 297 ---------------
.../examples/terasort/TestTeraSort.java | 94 -----
13 files changed, 3 insertions(+), 2680 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ff53e04..87729b3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -5,6 +5,7 @@ Release 0.6.0: Unreleased
INCOMPATIBLE CHANGES
TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
+ TEZ-1578. Remove TeraSort from Tez codebase.
ALL CHANGES:
TEZ-1544. Link to release artifacts for 0.5.0 does not point to a specific link for 0.5.0.
@@ -12,6 +13,8 @@ ALL CHANGES:
TEZ-850. Recovery unit tests.
TEZ-853. Support counters recovery.
TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized.
+ TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts.
+ TEZ-1578. Remove TeraSort from Tez codebase.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
index 977b767..3824607 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
@@ -31,9 +31,6 @@ import org.apache.tez.dag.api.client.DAGStatus;
import org.apache.tez.dag.api.client.Progress;
import org.apache.tez.dag.api.client.StatusGetOpts;
import org.apache.tez.dag.api.client.VertexStatus;
-import org.apache.tez.mapreduce.examples.terasort.TeraGen;
-import org.apache.tez.mapreduce.examples.terasort.TeraSort;
-import org.apache.tez.mapreduce.examples.terasort.TeraValidate;
/**
* A description of an example program based on its class and a
@@ -65,12 +62,6 @@ public class ExampleDriver {
pgd.addClass("join", Join.class,
"A job that effects a join over sorted, equally partitioned"
+ " datasets");
- pgd.addClass("teragen", TeraGen.class,
- "Generate data for the terasort");
- pgd.addClass("terasort", TeraSort.class,
- "Run the terasort");
- pgd.addClass("teravalidate", TeraValidate.class,
- "Checking results of terasort");
pgd.addClass("groupbyorderbymrrtest", GroupByOrderByMRRTest.class,
"A map-reduce-reduce program that does groupby-order by. Takes input"
+ " containing employee_name department name per line of input"
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
deleted file mode 100644
index c8517e5..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.math.BigInteger;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.util.PureJavaCrc32;
-
-/**
- * A single process data generator for the terasort data. Based on gensort.c
- * version 1.1 (3 Mar 2009) from Chris Nyberg <ch...@ordinal.com>.
- */
-public class GenSort {
-
- /**
- * Generate a "binary" record suitable for all sort benchmarks *except*
- * PennySort.
- */
- static void generateRecord(byte[] recBuf, Unsigned16 rand,
- Unsigned16 recordNumber) {
- /* generate the 10-byte key using the high 10 bytes of the 128-bit
- * random number
- */
- for(int i=0; i < 10; ++i) {
- recBuf[i] = rand.getByte(i);
- }
-
- /* add 2 bytes of "break" */
- recBuf[10] = 0x00;
- recBuf[11] = 0x11;
-
- /* convert the 128-bit record number to 32 bits of ascii hexadecimal
- * as the next 32 bytes of the record.
- */
- for (int i = 0; i < 32; i++) {
- recBuf[12 + i] = (byte) recordNumber.getHexDigit(i);
- }
-
- /* add 4 bytes of "break" data */
- recBuf[44] = (byte) 0x88;
- recBuf[45] = (byte) 0x99;
- recBuf[46] = (byte) 0xAA;
- recBuf[47] = (byte) 0xBB;
-
- /* add 48 bytes of filler based on low 48 bits of random number */
- for(int i=0; i < 12; ++i) {
- recBuf[48+i*4] = recBuf[49+i*4] = recBuf[50+i*4] = recBuf[51+i*4] =
- (byte) rand.getHexDigit(20 + i);
- }
-
- /* add 4 bytes of "break" data */
- recBuf[96] = (byte) 0xCC;
- recBuf[97] = (byte) 0xDD;
- recBuf[98] = (byte) 0xEE;
- recBuf[99] = (byte) 0xFF;
- }
-
-
- private static BigInteger makeBigInteger(long x) {
- byte[] data = new byte[8];
- for(int i=0; i < 8; ++i) {
- data[i] = (byte) (x >>> (56 - 8*i));
- }
- return new BigInteger(1, data);
- }
-
- private static final BigInteger NINETY_FIVE = new BigInteger("95");
-
- /**
- * Generate an ascii record suitable for all sort benchmarks including
- * PennySort.
- */
- static void generateAsciiRecord(byte[] recBuf, Unsigned16 rand,
- Unsigned16 recordNumber) {
-
- /* generate the 10-byte ascii key using mostly the high 64 bits.
- */
- long temp = rand.getHigh8();
- if (temp < 0) {
- // use biginteger to avoid the negative sign problem
- BigInteger bigTemp = makeBigInteger(temp);
- recBuf[0] = (byte) (' ' + (bigTemp.mod(NINETY_FIVE).longValue()));
- temp = bigTemp.divide(NINETY_FIVE).longValue();
- } else {
- recBuf[0] = (byte) (' ' + (temp % 95));
- temp /= 95;
- }
- for(int i=1; i < 8; ++i) {
- recBuf[i] = (byte) (' ' + (temp % 95));
- temp /= 95;
- }
- temp = rand.getLow8();
- if (temp < 0) {
- BigInteger bigTemp = makeBigInteger(temp);
- recBuf[8] = (byte) (' ' + (bigTemp.mod(NINETY_FIVE).longValue()));
- temp = bigTemp.divide(NINETY_FIVE).longValue();
- } else {
- recBuf[8] = (byte) (' ' + (temp % 95));
- temp /= 95;
- }
- recBuf[9] = (byte)(' ' + (temp % 95));
-
- /* add 2 bytes of "break" */
- recBuf[10] = ' ';
- recBuf[11] = ' ';
-
- /* convert the 128-bit record number to 32 bits of ascii hexadecimal
- * as the next 32 bytes of the record.
- */
- for (int i = 0; i < 32; i++) {
- recBuf[12 + i] = (byte) recordNumber.getHexDigit(i);
- }
-
- /* add 2 bytes of "break" data */
- recBuf[44] = ' ';
- recBuf[45] = ' ';
-
- /* add 52 bytes of filler based on low 48 bits of random number */
- for(int i=0; i < 13; ++i) {
- recBuf[46+i*4] = recBuf[47+i*4] = recBuf[48+i*4] = recBuf[49+i*4] =
- (byte) rand.getHexDigit(19 + i);
- }
-
- /* add 2 bytes of "break" data */
- recBuf[98] = '\r'; /* nice for Windows */
- recBuf[99] = '\n';
-}
-
-
- private static void usage() {
- PrintStream out = System.out;
- out.println("usage: gensort [-a] [-c] [-bSTARTING_REC_NUM] NUM_RECS FILE_NAME");
- out.println("-a Generate ascii records required for PennySort or JouleSort.");
- out.println(" These records are also an alternative input for the other");
- out.println(" sort benchmarks. Without this flag, binary records will be");
- out.println(" generated that contain the highest density of randomness in");
- out.println(" the 10-byte key.");
- out.println( "-c Calculate the sum of the crc32 checksums of each of the");
- out.println(" generated records and send it to standard error.");
- out.println("-bN Set the beginning record generated to N. By default the");
- out.println(" first record generated is record 0.");
- out.println("NUM_RECS The number of sequential records to generate.");
- out.println("FILE_NAME The name of the file to write the records to.\n");
- out.println("Example 1 - to generate 1000000 ascii records starting at record 0 to");
- out.println("the file named \"pennyinput\":");
- out.println(" gensort -a 1000000 pennyinput\n");
- out.println("Example 2 - to generate 1000 binary records beginning with record 2000");
- out.println("to the file named \"partition2\":");
- out.println(" gensort -b2000 1000 partition2");
- System.exit(1);
- }
-
-
- public static void outputRecords(OutputStream out,
- boolean useAscii,
- Unsigned16 firstRecordNumber,
- Unsigned16 recordsToGenerate,
- Unsigned16 checksum
- ) throws IOException {
- byte[] row = new byte[100];
- Unsigned16 recordNumber = new Unsigned16(firstRecordNumber);
- Unsigned16 lastRecordNumber = new Unsigned16(firstRecordNumber);
- Checksum crc = new PureJavaCrc32();
- Unsigned16 tmp = new Unsigned16();
- lastRecordNumber.add(recordsToGenerate);
- Unsigned16 ONE = new Unsigned16(1);
- Unsigned16 rand = Random16.skipAhead(firstRecordNumber);
- while (!recordNumber.equals(lastRecordNumber)) {
- Random16.nextRand(rand);
- if (useAscii) {
- generateAsciiRecord(row, rand, recordNumber);
- } else {
- generateRecord(row, rand, recordNumber);
- }
- if (checksum != null) {
- crc.reset();
- crc.update(row, 0, row.length);
- tmp.set(crc.getValue());
- checksum.add(tmp);
- }
- recordNumber.add(ONE);
- out.write(row);
- }
- }
-
- public static void main(String[] args) throws Exception {
- Unsigned16 startingRecord = new Unsigned16();
- Unsigned16 numberOfRecords;
- OutputStream out;
- boolean useAscii = false;
- Unsigned16 checksum = null;
-
- int i;
- for(i=0; i < args.length; ++i) {
- String arg = args[i];
- int argLength = arg.length();
- if (argLength >= 1 && arg.charAt(0) == '-') {
- if (argLength < 2) {
- usage();
- }
- switch (arg.charAt(1)) {
- case 'a':
- useAscii = true;
- break;
- case 'b':
- startingRecord = Unsigned16.fromDecimal(arg.substring(2));
- break;
- case 'c':
- checksum = new Unsigned16();
- break;
- default:
- usage();
- }
- } else {
- break;
- }
- }
- if (args.length - i != 2) {
- usage();
- }
- numberOfRecords = Unsigned16.fromDecimal(args[i]);
- out = new FileOutputStream(args[i+1]);
-
- outputRecords(out, useAscii, startingRecord, numberOfRecords, checksum);
- out.close();
- if (checksum != null) {
- System.out.println(checksum);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
deleted file mode 100644
index 31cbd48..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
+++ /dev/null
@@ -1,374 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.mapreduce.examples.terasort;
-
-/**
- * This class implements a 128-bit linear congruential generator.
- * Specifically, if X0 is the most recently issued 128-bit random
- * number (or a seed of 0 if no random number has already been generated,
- * the next number to be generated, X1, is equal to:
- * X1 = (a * X0 + c) mod 2**128
- * where a is 47026247687942121848144207491837523525
- * or 0x2360ed051fc65da44385df649fccf645
- * and c is 98910279301475397889117759788405497857
- * or 0x4a696d47726179524950202020202001
- * The coefficient "a" is suggested by:
- * Pierre L'Ecuyer, "Tables of linear congruential generators of different
- * sizes and good lattice structure", Mathematics of Computation, 68
- * pp. 249 - 260 (1999)
- * http://www.ams.org/mcom/1999-68-225/S0025-5718-99-00996-5/S0025-5718-99-00996-5.pdf
- * The constant "c" meets the simple suggestion by the same reference that
- * it be odd.
- *
- * There is also a facility for quickly advancing the state of the
- * generator by a fixed number of steps - this facilitates parallel
- * generation.
- *
- * This is based on 1.0 of rand16.c from Chris Nyberg
- * <ch...@ordinal.com>.
- */
-class Random16 {
-
- /**
- * The "Gen" array contain powers of 2 of the linear congruential generator.
- * The index 0 struct contain the "a" coefficient and "c" constant for the
- * generator. That is, the generator is:
- * f(x) = (Gen[0].a * x + Gen[0].c) mod 2**128
- *
- * All structs after the first contain an "a" and "c" that
- * comprise the square of the previous function.
- *
- * f**2(x) = (Gen[1].a * x + Gen[1].c) mod 2**128
- * f**4(x) = (Gen[2].a * x + Gen[2].c) mod 2**128
- * f**8(x) = (Gen[3].a * x + Gen[3].c) mod 2**128
- * ...
-
- */
- private static class RandomConstant {
- final Unsigned16 a;
- final Unsigned16 c;
- public RandomConstant(String left, String right) {
- a = new Unsigned16(left);
- c = new Unsigned16(right);
- }
- }
-
- private static final RandomConstant[] genArray = new RandomConstant[]{
- /* [ 0] */ new RandomConstant("2360ed051fc65da44385df649fccf645",
- "4a696d47726179524950202020202001"),
- /* [ 1] */ new RandomConstant("17bce35bdf69743c529ed9eb20e0ae99",
- "95e0e48262b3edfe04479485c755b646"),
- /* [ 2] */ new RandomConstant("f4dd417327db7a9bd194dfbe42d45771",
- "882a02c315362b60765f100068b33a1c"),
- /* [ 3] */ new RandomConstant("6347af777a7898f6d1a2d6f33505ffe1",
- "5efc4abfaca23e8ca8edb1f2dfbf6478"),
- /* [ 4] */ new RandomConstant("b6a4239f3b315f84f6ef6d3d288c03c1",
- "f25bd15439d16af594c1b1bafa6239f0"),
- /* [ 5] */ new RandomConstant("2c82901ad1cb0cd182b631ba6b261781",
- "89ca67c29c9397d59c612596145db7e0"),
- /* [ 6] */ new RandomConstant("dab03f988288676ee49e66c4d2746f01",
- "8b6ae036713bd578a8093c8eae5c7fc0"),
- /* [ 7] */ new RandomConstant("602167331d86cf5684fe009a6d09de01",
- "98a2542fd23d0dbdff3b886cdb1d3f80"),
- /* [ 8] */ new RandomConstant("61ecb5c24d95b058f04c80a23697bc01",
- "954db923fdb7933e947cd1edcecb7f00"),
- /* [ 9] */ new RandomConstant("4a5c31e0654c28aa60474e83bf3f7801",
- "00be4a36657c98cd204e8c8af7dafe00"),
- /* [ 10] */ new RandomConstant("ae4f079d54fbece1478331d3c6bef001",
- "991965329dccb28d581199ab18c5fc00"),
- /* [ 11] */ new RandomConstant("101b8cb830c7cb927ff1ed50ae7de001",
- "e1a8705b63ad5b8cd6c3d268d5cbf800"),
- /* [ 12] */ new RandomConstant("f54a27fc056b00e7563f3505e0fbc001",
- "2b657bbfd6ed9d632079e70c3c97f000"),
- /* [ 13] */ new RandomConstant("df8a6fc1a833d201f98d719dd1f78001",
- "59b60ee4c52fa49e9fe90682bd2fe000"),
- /* [ 14] */ new RandomConstant("5480a5015f101a4ea7e3f183e3ef0001",
- "cc099c88030679464fe86aae8a5fc000"),
- /* [ 15] */ new RandomConstant("a498509e76e5d7925f539c28c7de0001",
- "06b9abff9f9f33dd30362c0154bf8000"),
- /* [ 16] */ new RandomConstant("0798a3d8b10dc72e60121cd58fbc0001",
- "e296707121688d5a0260b293a97f0000"),
- /* [ 17] */ new RandomConstant("1647d1e78ec02e665fafcbbb1f780001",
- "189ffc4701ff23cb8f8acf6b52fe0000"),
- /* [ 18] */ new RandomConstant("a7c982285e72bf8c0c8ddfb63ef00001",
- "5141110ab208fb9d61fb47e6a5fc0000"),
- /* [ 19] */ new RandomConstant("3eb78ee8fb8c56dbc5d4e06c7de00001",
- "3c97caa62540f2948d8d340d4bf80000"),
- /* [ 20] */ new RandomConstant("72d03b6f4681f2f9fe8e44d8fbc00001",
- "1b25cb9cfe5a0c963174f91a97f00000"),
- /* [ 21] */ new RandomConstant("ea85f81e4f502c9bc8ae99b1f7800001",
- "0c644570b4a487103c5436352fe00000"),
- /* [ 22] */ new RandomConstant("629c320db08b00c6bfa57363ef000001",
- "3d0589c28869472bde517c6a5fc00000"),
- /* [ 23] */ new RandomConstant("c5c4b9ce268d074a386be6c7de000001",
- "bc95e5ab36477e65534738d4bf800000"),
- /* [ 24] */ new RandomConstant("f30bbbbed1596187555bcd8fbc000001",
- "ddb02ff72a031c01011f71a97f000000"),
- /* [ 25] */ new RandomConstant("4a1000fb26c9eeda3cc79b1f78000001",
- "2561426086d9acdb6c82e352fe000000"),
- /* [ 26] */ new RandomConstant("89fb5307f6bf8ce2c1cf363ef0000001",
- "64a788e3c118ed1c8215c6a5fc000000"),
- /* [ 27] */ new RandomConstant("830b7b3358a5d67ea49e6c7de0000001",
- "e65ea321908627cfa86b8d4bf8000000"),
- /* [ 28] */ new RandomConstant("fd8a51da91a69fe1cd3cd8fbc0000001",
- "53d27225604d85f9e1d71a97f0000000"),
- /* [ 29] */ new RandomConstant("901a48b642b90b55aa79b1f780000001",
- "ca5ec7a3ed1fe55e07ae352fe0000000"),
- /* [ 30] */ new RandomConstant("118cdefdf32144f394f363ef00000001",
- "4daebb2e085330651f5c6a5fc0000000"),
- /* [ 31] */ new RandomConstant("0a88c0a91cff430829e6c7de00000001",
- "9d6f1a00a8f3f76e7eb8d4bf80000000"),
- /* [ 32] */ new RandomConstant("433bef4314f16a9453cd8fbc00000001",
- "158c62f2b31e496dfd71a97f00000000"),
- /* [ 33] */ new RandomConstant("c294b02995ae6738a79b1f7800000001",
- "290e84a2eb15fd1ffae352fe00000000"),
- /* [ 34] */ new RandomConstant("913575e0da8b16b14f363ef000000001",
- "e3dc1bfbe991a34ff5c6a5fc00000000"),
- /* [ 35] */ new RandomConstant("2f61b9f871cf4e629e6c7de000000001",
- "ddf540d020b9eadfeb8d4bf800000000"),
- /* [ 36] */ new RandomConstant("78d26ccbd68320c53cd8fbc000000001",
- "8ee4950177ce66bfd71a97f000000000"),
- /* [ 37] */ new RandomConstant("8b7ebd037898518a79b1f78000000001",
- "39e0f787c907117fae352fe000000000"),
- /* [ 38] */ new RandomConstant("0b5507b61f78e314f363ef0000000001",
- "659d2522f7b732ff5c6a5fc000000000"),
- /* [ 39] */ new RandomConstant("4f884628f812c629e6c7de0000000001",
- "9e8722938612a5feb8d4bf8000000000"),
- /* [ 40] */ new RandomConstant("be896744d4a98c53cd8fbc0000000001",
- "e941a65d66b64bfd71a97f0000000000"),
- /* [ 41] */ new RandomConstant("daf63a553b6318a79b1f780000000001",
- "7b50d19437b097fae352fe0000000000"),
- /* [ 42] */ new RandomConstant("2d7a23d8bf06314f363ef00000000001",
- "59d7b68e18712ff5c6a5fc0000000000"),
- /* [ 43] */ new RandomConstant("392b046a9f0c629e6c7de00000000001",
- "4087bab2d5225feb8d4bf80000000000"),
- /* [ 44] */ new RandomConstant("eb30fbb9c218c53cd8fbc00000000001",
- "b470abc03b44bfd71a97f00000000000"),
- /* [ 45] */ new RandomConstant("b9cdc30594318a79b1f7800000000001",
- "366630eaba897fae352fe00000000000"),
- /* [ 46] */ new RandomConstant("014ab453686314f363ef000000000001",
- "a2dfc77e8512ff5c6a5fc00000000000"),
- /* [ 47] */ new RandomConstant("395221c7d0c629e6c7de000000000001",
- "1e0d25a14a25feb8d4bf800000000000"),
- /* [ 48] */ new RandomConstant("4d972813a18c53cd8fbc000000000001",
- "9d50a5d3944bfd71a97f000000000000"),
- /* [ 49] */ new RandomConstant("06f9e2374318a79b1f78000000000001",
- "bf7ab5eb2897fae352fe000000000000"),
- /* [ 50] */ new RandomConstant("bd220cae86314f363ef0000000000001",
- "925b14e6512ff5c6a5fc000000000000"),
- /* [ 51] */ new RandomConstant("36fd3a5d0c629e6c7de0000000000001",
- "724cce0ca25feb8d4bf8000000000000"),
- /* [ 52] */ new RandomConstant("60def8ba18c53cd8fbc0000000000001",
- "1af42d1944bfd71a97f0000000000000"),
- /* [ 53] */ new RandomConstant("8d500174318a79b1f780000000000001",
- "0f529e32897fae352fe0000000000000"),
- /* [ 54] */ new RandomConstant("48e842e86314f363ef00000000000001",
- "844e4c6512ff5c6a5fc0000000000000"),
- /* [ 55] */ new RandomConstant("4af185d0c629e6c7de00000000000001",
- "9f40d8ca25feb8d4bf80000000000000"),
- /* [ 56] */ new RandomConstant("7a670ba18c53cd8fbc00000000000001",
- "9912b1944bfd71a97f00000000000000"),
- /* [ 57] */ new RandomConstant("86de174318a79b1f7800000000000001",
- "9c69632897fae352fe00000000000000"),
- /* [ 58] */ new RandomConstant("55fc2e86314f363ef000000000000001",
- "e1e2c6512ff5c6a5fc00000000000000"),
- /* [ 59] */ new RandomConstant("ccf85d0c629e6c7de000000000000001",
- "68058ca25feb8d4bf800000000000000"),
- /* [ 60] */ new RandomConstant("1df0ba18c53cd8fbc000000000000001",
- "610b1944bfd71a97f000000000000000"),
- /* [ 61] */ new RandomConstant("4be174318a79b1f78000000000000001",
- "061632897fae352fe000000000000000"),
- /* [ 62] */ new RandomConstant("d7c2e86314f363ef0000000000000001",
- "1c2c6512ff5c6a5fc000000000000000"),
- /* [ 63] */ new RandomConstant("af85d0c629e6c7de0000000000000001",
- "7858ca25feb8d4bf8000000000000000"),
- /* [ 64] */ new RandomConstant("5f0ba18c53cd8fbc0000000000000001",
- "f0b1944bfd71a97f0000000000000000"),
- /* [ 65] */ new RandomConstant("be174318a79b1f780000000000000001",
- "e1632897fae352fe0000000000000000"),
- /* [ 66] */ new RandomConstant("7c2e86314f363ef00000000000000001",
- "c2c6512ff5c6a5fc0000000000000000"),
- /* [ 67] */ new RandomConstant("f85d0c629e6c7de00000000000000001",
- "858ca25feb8d4bf80000000000000000"),
- /* [ 68] */ new RandomConstant("f0ba18c53cd8fbc00000000000000001",
- "0b1944bfd71a97f00000000000000000"),
- /* [ 69] */ new RandomConstant("e174318a79b1f7800000000000000001",
- "1632897fae352fe00000000000000000"),
- /* [ 70] */ new RandomConstant("c2e86314f363ef000000000000000001",
- "2c6512ff5c6a5fc00000000000000000"),
- /* [ 71] */ new RandomConstant("85d0c629e6c7de000000000000000001",
- "58ca25feb8d4bf800000000000000000"),
- /* [ 72] */ new RandomConstant("0ba18c53cd8fbc000000000000000001",
- "b1944bfd71a97f000000000000000000"),
- /* [ 73] */ new RandomConstant("174318a79b1f78000000000000000001",
- "632897fae352fe000000000000000000"),
- /* [ 74] */ new RandomConstant("2e86314f363ef0000000000000000001",
- "c6512ff5c6a5fc000000000000000000"),
- /* [ 75] */ new RandomConstant("5d0c629e6c7de0000000000000000001",
- "8ca25feb8d4bf8000000000000000000"),
- /* [ 76] */ new RandomConstant("ba18c53cd8fbc0000000000000000001",
- "1944bfd71a97f0000000000000000000"),
- /* [ 77] */ new RandomConstant("74318a79b1f780000000000000000001",
- "32897fae352fe0000000000000000000"),
- /* [ 78] */ new RandomConstant("e86314f363ef00000000000000000001",
- "6512ff5c6a5fc0000000000000000000"),
- /* [ 79] */ new RandomConstant("d0c629e6c7de00000000000000000001",
- "ca25feb8d4bf80000000000000000000"),
- /* [ 80] */ new RandomConstant("a18c53cd8fbc00000000000000000001",
- "944bfd71a97f00000000000000000000"),
- /* [ 81] */ new RandomConstant("4318a79b1f7800000000000000000001",
- "2897fae352fe00000000000000000000"),
- /* [ 82] */ new RandomConstant("86314f363ef000000000000000000001",
- "512ff5c6a5fc00000000000000000000"),
- /* [ 83] */ new RandomConstant("0c629e6c7de000000000000000000001",
- "a25feb8d4bf800000000000000000000"),
- /* [ 84] */ new RandomConstant("18c53cd8fbc000000000000000000001",
- "44bfd71a97f000000000000000000000"),
- /* [ 85] */ new RandomConstant("318a79b1f78000000000000000000001",
- "897fae352fe000000000000000000000"),
- /* [ 86] */ new RandomConstant("6314f363ef0000000000000000000001",
- "12ff5c6a5fc000000000000000000000"),
- /* [ 87] */ new RandomConstant("c629e6c7de0000000000000000000001",
- "25feb8d4bf8000000000000000000000"),
- /* [ 88] */ new RandomConstant("8c53cd8fbc0000000000000000000001",
- "4bfd71a97f0000000000000000000000"),
- /* [ 89] */ new RandomConstant("18a79b1f780000000000000000000001",
- "97fae352fe0000000000000000000000"),
- /* [ 90] */ new RandomConstant("314f363ef00000000000000000000001",
- "2ff5c6a5fc0000000000000000000000"),
- /* [ 91] */ new RandomConstant("629e6c7de00000000000000000000001",
- "5feb8d4bf80000000000000000000000"),
- /* [ 92] */ new RandomConstant("c53cd8fbc00000000000000000000001",
- "bfd71a97f00000000000000000000000"),
- /* [ 93] */ new RandomConstant("8a79b1f7800000000000000000000001",
- "7fae352fe00000000000000000000000"),
- /* [ 94] */ new RandomConstant("14f363ef000000000000000000000001",
- "ff5c6a5fc00000000000000000000000"),
- /* [ 95] */ new RandomConstant("29e6c7de000000000000000000000001",
- "feb8d4bf800000000000000000000000"),
- /* [ 96] */ new RandomConstant("53cd8fbc000000000000000000000001",
- "fd71a97f000000000000000000000000"),
- /* [ 97] */ new RandomConstant("a79b1f78000000000000000000000001",
- "fae352fe000000000000000000000000"),
- /* [ 98] */ new RandomConstant("4f363ef0000000000000000000000001",
- "f5c6a5fc000000000000000000000000"),
- /* [ 99] */ new RandomConstant("9e6c7de0000000000000000000000001",
- "eb8d4bf8000000000000000000000000"),
- /* [100] */ new RandomConstant("3cd8fbc0000000000000000000000001",
- "d71a97f0000000000000000000000000"),
- /* [101] */ new RandomConstant("79b1f780000000000000000000000001",
- "ae352fe0000000000000000000000000"),
- /* [102] */ new RandomConstant("f363ef00000000000000000000000001",
- "5c6a5fc0000000000000000000000000"),
- /* [103] */ new RandomConstant("e6c7de00000000000000000000000001",
- "b8d4bf80000000000000000000000000"),
- /* [104] */ new RandomConstant("cd8fbc00000000000000000000000001",
- "71a97f00000000000000000000000000"),
- /* [105] */ new RandomConstant("9b1f7800000000000000000000000001",
- "e352fe00000000000000000000000000"),
- /* [106] */ new RandomConstant("363ef000000000000000000000000001",
- "c6a5fc00000000000000000000000000"),
- /* [107] */ new RandomConstant("6c7de000000000000000000000000001",
- "8d4bf800000000000000000000000000"),
- /* [108] */ new RandomConstant("d8fbc000000000000000000000000001",
- "1a97f000000000000000000000000000"),
- /* [109] */ new RandomConstant("b1f78000000000000000000000000001",
- "352fe000000000000000000000000000"),
- /* [110] */ new RandomConstant("63ef0000000000000000000000000001",
- "6a5fc000000000000000000000000000"),
- /* [111] */ new RandomConstant("c7de0000000000000000000000000001",
- "d4bf8000000000000000000000000000"),
- /* [112] */ new RandomConstant("8fbc0000000000000000000000000001",
- "a97f0000000000000000000000000000"),
- /* [113] */ new RandomConstant("1f780000000000000000000000000001",
- "52fe0000000000000000000000000000"),
- /* [114] */ new RandomConstant("3ef00000000000000000000000000001",
- "a5fc0000000000000000000000000000"),
- /* [115] */ new RandomConstant("7de00000000000000000000000000001",
- "4bf80000000000000000000000000000"),
- /* [116] */ new RandomConstant("fbc00000000000000000000000000001",
- "97f00000000000000000000000000000"),
- /* [117] */ new RandomConstant("f7800000000000000000000000000001",
- "2fe00000000000000000000000000000"),
- /* [118] */ new RandomConstant("ef000000000000000000000000000001",
- "5fc00000000000000000000000000000"),
- /* [119] */ new RandomConstant("de000000000000000000000000000001",
- "bf800000000000000000000000000000"),
- /* [120] */ new RandomConstant("bc000000000000000000000000000001",
- "7f000000000000000000000000000000"),
- /* [121] */ new RandomConstant("78000000000000000000000000000001",
- "fe000000000000000000000000000000"),
- /* [122] */ new RandomConstant("f0000000000000000000000000000001",
- "fc000000000000000000000000000000"),
- /* [123] */ new RandomConstant("e0000000000000000000000000000001",
- "f8000000000000000000000000000000"),
- /* [124] */ new RandomConstant("c0000000000000000000000000000001",
- "f0000000000000000000000000000000"),
- /* [125] */ new RandomConstant("80000000000000000000000000000001",
- "e0000000000000000000000000000000"),
- /* [126] */ new RandomConstant("00000000000000000000000000000001",
- "c0000000000000000000000000000000"),
- /* [127] */ new RandomConstant("00000000000000000000000000000001",
- "80000000000000000000000000000000")};
-
- /**
- * generate the random number that is "advance" steps
- * from an initial random number of 0. This is done by
- * starting with 0, and then advancing the by the
- * appropriate powers of 2 of the linear congruential
- * generator.
- */
- public static Unsigned16 skipAhead(Unsigned16 advance) {
- Unsigned16 result = new Unsigned16();
- long bit_map;
-
- bit_map = advance.getLow8();
- for (int i = 0; bit_map != 0 && i < 64; i++) {
- if ((bit_map & (1L << i)) != 0) {
- /* advance random number by f**(2**i) (x)
- */
- result.multiply(genArray[i].a);
- result.add(genArray[i].c);
- bit_map &= ~(1L << i);
- }
- }
- bit_map = advance.getHigh8();
- for (int i = 0; bit_map != 0 && i < 64; i++)
- {
- if ((bit_map & (1L << i)) != 0) {
- /* advance random number by f**(2**(i + 64)) (x)
- */
- result.multiply(genArray[i+64].a);
- result.add(genArray[i+64].c);
- bit_map &= ~(1L << i);
- }
- }
- return result;
- }
-
- /**
- * Generate the next 16 byte random number.
- */
- public static void nextRand(Unsigned16 rand) {
- /* advance the random number forward once using the linear congruential
- * generator, and then return the new random number
- */
- rand.multiply(genArray[0].a);
- rand.add(genArray[0].c);
- }
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
deleted file mode 100644
index a5b408b..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.IOException;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-public class TeraChecksum extends Configured implements Tool {
- static class ChecksumMapper
- extends Mapper<Text, Text, NullWritable, Unsigned16> {
- private Unsigned16 checksum = new Unsigned16();
- private Unsigned16 sum = new Unsigned16();
- private Checksum crc32 = new PureJavaCrc32();
-
- public void map(Text key, Text value,
- Context context) throws IOException {
- crc32.reset();
- crc32.update(key.getBytes(), 0, key.getLength());
- crc32.update(value.getBytes(), 0, value.getLength());
- checksum.set(crc32.getValue());
- sum.add(checksum);
- }
-
- public void cleanup(Context context)
- throws IOException, InterruptedException {
- context.write(NullWritable.get(), sum);
- }
- }
-
- static class ChecksumReducer
- extends Reducer<NullWritable, Unsigned16, NullWritable, Unsigned16> {
-
- public void reduce(NullWritable key, Iterable<Unsigned16> values,
- Context context) throws IOException, InterruptedException {
- Unsigned16 sum = new Unsigned16();
- for (Unsigned16 val : values) {
- sum.add(val);
- }
- context.write(key, sum);
- }
- }
-
- private static void usage() throws IOException {
- System.err.println("terasum <out-dir> <report-dir>");
- }
-
- public int run(String[] args) throws Exception {
- Job job = Job.getInstance(getConf());
- if (args.length != 2) {
- usage();
- return 2;
- }
- TeraInputFormat.setInputPaths(job, new Path(args[0]));
- FileOutputFormat.setOutputPath(job, new Path(args[1]));
- job.setJobName("TeraSum");
- job.setJarByClass(TeraChecksum.class);
- job.setMapperClass(ChecksumMapper.class);
- job.setReducerClass(ChecksumReducer.class);
- job.setOutputKeyClass(NullWritable.class);
- job.setOutputValueClass(Unsigned16.class);
- // force a single reducer
- job.setNumReduceTasks(1);
- job.setInputFormatClass(TeraInputFormat.class);
- return job.waitForCompletion(true) ? 0 : 1;
- }
-
- /**
- * @param args
- */
- public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new Configuration(), new TeraChecksum(), args);
- System.exit(res);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
deleted file mode 100644
index 70a4207..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
+++ /dev/null
@@ -1,311 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.zip.Checksum;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generate the official GraySort input data set.
- * The user specifies the number of rows and the output directory and this
- * class runs a map/reduce program to generate the data.
- * The format of the data is:
- * <ul>
- * <li>(10 bytes key) (constant 2 bytes) (32 bytes rowid)
- * (constant 4 bytes) (48 bytes filler) (constant 4 bytes)
- * <li>The rowid is the right justified row id as a hex number.
- * </ul>
- *
- * <p>
- * To run the program:
- * <b>bin/hadoop jar hadoop-*-examples.jar teragen 10000000000 in-dir</b>
- */
-public class TeraGen extends Configured implements Tool {
- private static final Log LOG = LogFactory.getLog(TeraSort.class);
-
- public static enum Counters {CHECKSUM}
-
- public static final String NUM_ROWS = "mapreduce.terasort.num-rows";
- /**
- * An input format that assigns ranges of longs to each mapper.
- */
- static class RangeInputFormat
- extends InputFormat<LongWritable, NullWritable> {
-
- /**
- * An input split consisting of a range on numbers.
- */
- static class RangeInputSplit extends InputSplit implements Writable {
- long firstRow;
- long rowCount;
-
- public RangeInputSplit() { }
-
- public RangeInputSplit(long offset, long length) {
- firstRow = offset;
- rowCount = length;
- }
-
- public long getLength() throws IOException {
- return 0;
- }
-
- public String[] getLocations() throws IOException {
- return new String[]{};
- }
-
- public void readFields(DataInput in) throws IOException {
- firstRow = WritableUtils.readVLong(in);
- rowCount = WritableUtils.readVLong(in);
- }
-
- public void write(DataOutput out) throws IOException {
- WritableUtils.writeVLong(out, firstRow);
- WritableUtils.writeVLong(out, rowCount);
- }
- }
-
- /**
- * A record reader that will generate a range of numbers.
- */
- static class RangeRecordReader
- extends RecordReader<LongWritable, NullWritable> {
- long startRow;
- long finishedRows;
- long totalRows;
- LongWritable key = null;
-
- public RangeRecordReader() {
- }
-
- public void initialize(InputSplit split, TaskAttemptContext context)
- throws IOException, InterruptedException {
- startRow = ((RangeInputSplit)split).firstRow;
- finishedRows = 0;
- totalRows = ((RangeInputSplit)split).rowCount;
- }
-
- public void close() throws IOException {
- // NOTHING
- }
-
- public LongWritable getCurrentKey() {
- return key;
- }
-
- public NullWritable getCurrentValue() {
- return NullWritable.get();
- }
-
- public float getProgress() throws IOException {
- return finishedRows / (float) totalRows;
- }
-
- public boolean nextKeyValue() {
- if (key == null) {
- key = new LongWritable();
- }
- if (finishedRows < totalRows) {
- key.set(startRow + finishedRows);
- finishedRows += 1;
- return true;
- } else {
- return false;
- }
- }
-
- }
-
- public RecordReader<LongWritable, NullWritable>
- createRecordReader(InputSplit split, TaskAttemptContext context)
- throws IOException {
- return new RangeRecordReader();
- }
-
- /**
- * Create the desired number of splits, dividing the number of rows
- * between the mappers.
- */
- public List<InputSplit> getSplits(JobContext job) {
- long totalRows = getNumberOfRows(job);
- int numSplits = job.getConfiguration().getInt(MRJobConfig.NUM_MAPS, 1);
- LOG.info("Generating " + totalRows + " using " + numSplits);
- List<InputSplit> splits = new ArrayList<InputSplit>();
- long currentRow = 0;
- for(int split = 0; split < numSplits; ++split) {
- long goal =
- (long) Math.ceil(totalRows * (double)(split + 1) / numSplits);
- splits.add(new RangeInputSplit(currentRow, goal - currentRow));
- currentRow = goal;
- }
- return splits;
- }
-
- }
-
- static long getNumberOfRows(JobContext job) {
- return job.getConfiguration().getLong(NUM_ROWS, 0);
- }
-
- static void setNumberOfRows(Job job, long numRows) {
- job.getConfiguration().setLong(NUM_ROWS, numRows);
- }
-
- /**
- * The Mapper class that given a row number, will generate the appropriate
- * output line.
- */
- public static class SortGenMapper
- extends Mapper<LongWritable, NullWritable, Text, Text> {
-
- private Text key = new Text();
- private Text value = new Text();
- private Unsigned16 rand = null;
- private Unsigned16 rowId = null;
- private Unsigned16 checksum = new Unsigned16();
- private Checksum crc32 = new PureJavaCrc32();
- private Unsigned16 total = new Unsigned16();
- private static final Unsigned16 ONE = new Unsigned16(1);
- private byte[] buffer = new byte[TeraInputFormat.KEY_LENGTH +
- TeraInputFormat.VALUE_LENGTH];
- private Counter checksumCounter;
-
- public void map(LongWritable row, NullWritable ignored,
- Context context) throws IOException, InterruptedException {
- if (rand == null) {
- rowId = new Unsigned16(row.get());
- rand = Random16.skipAhead(rowId);
- checksumCounter = context.getCounter(Counters.CHECKSUM);
- }
- Random16.nextRand(rand);
- GenSort.generateRecord(buffer, rand, rowId);
- key.set(buffer, 0, TeraInputFormat.KEY_LENGTH);
- value.set(buffer, TeraInputFormat.KEY_LENGTH,
- TeraInputFormat.VALUE_LENGTH);
- context.write(key, value);
- crc32.reset();
- crc32.update(buffer, 0,
- TeraInputFormat.KEY_LENGTH + TeraInputFormat.VALUE_LENGTH);
- checksum.set(crc32.getValue());
- total.add(checksum);
- rowId.add(ONE);
- }
-
- @Override
- public void cleanup(Context context) {
- if (checksumCounter != null) {
- checksumCounter.increment(total.getLow8());
- }
- }
- }
-
- private static void usage() throws IOException {
- System.err.println("teragen <num rows> <output dir>");
- }
-
- /**
- * Parse a number that optionally has a postfix that denotes a base.
- * @param str an string integer with an option base {k,m,b,t}.
- * @return the expanded value
- */
- private static long parseHumanLong(String str) {
- char tail = str.charAt(str.length() - 1);
- long base = 1;
- switch (tail) {
- case 't':
- base *= 1000 * 1000 * 1000 * 1000;
- break;
- case 'b':
- base *= 1000 * 1000 * 1000;
- break;
- case 'm':
- base *= 1000 * 1000;
- break;
- case 'k':
- base *= 1000;
- break;
- default:
- }
- if (base != 1) {
- str = str.substring(0, str.length() - 1);
- }
- return Long.parseLong(str) * base;
- }
-
- /**
- * @param args the cli arguments
- */
- public int run(String[] args)
- throws IOException, InterruptedException, ClassNotFoundException {
- Job job = Job.getInstance(getConf());
- if (args.length != 2) {
- usage();
- return 2;
- }
- setNumberOfRows(job, parseHumanLong(args[0]));
- Path outputDir = new Path(args[1]);
- if (outputDir.getFileSystem(getConf()).exists(outputDir)) {
- throw new IOException("Output directory " + outputDir +
- " already exists.");
- }
- FileOutputFormat.setOutputPath(job, outputDir);
- job.setJobName("TeraGen");
- job.setJarByClass(TeraGen.class);
- job.setMapperClass(SortGenMapper.class);
- job.setNumReduceTasks(0);
- job.setOutputKeyClass(Text.class);
- job.setOutputValueClass(Text.class);
- job.setInputFormatClass(RangeInputFormat.class);
- job.setOutputFormatClass(TeraOutputFormat.class);
- return job.waitForCompletion(true) ? 0 : 1;
- }
-
- public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new Configuration(), new TeraGen(), args);
- System.exit(res);
- }
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
deleted file mode 100644
index 6c9b201..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
+++ /dev/null
@@ -1,353 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.hadoop.util.IndexedSortable;
-import org.apache.hadoop.util.QuickSort;
-
-/**
- * An input format that reads the first 10 characters of each line as the key
- * and the rest of the line as the value. Both key and value are represented
- * as Text.
- */
-public class TeraInputFormat extends FileInputFormat<Text,Text> {
-
- static final String PARTITION_FILENAME = "_partition.lst";
- private static final String NUM_PARTITIONS =
- "mapreduce.terasort.num.partitions";
- private static final String SAMPLE_SIZE =
- "mapreduce.terasort.partitions.sample";
- static final int KEY_LENGTH = 10;
- static final int VALUE_LENGTH = 90;
- static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
- private static MRJobConfig lastContext = null;
- private static List<InputSplit> lastResult = null;
-
- static class TeraFileSplit extends FileSplit {
- static private String[] ZERO_LOCATIONS = new String[0];
-
- private String[] locations;
-
- public TeraFileSplit() {
- locations = ZERO_LOCATIONS;
- }
- public TeraFileSplit(Path file, long start, long length, String[] hosts) {
- super(file, start, length, hosts);
- try {
- locations = super.getLocations();
- } catch (IOException e) {
- locations = ZERO_LOCATIONS;
- }
- }
-
- // XXXXXX should this also be null-protected?
- protected void setLocations(String[] hosts) {
- locations = hosts;
- }
-
- @Override
- public String[] getLocations() {
- return locations;
- }
-
- public String toString() {
- StringBuffer result = new StringBuffer();
- result.append(getPath());
- result.append(" from ");
- result.append(getStart());
- result.append(" length ");
- result.append(getLength());
- for(String host: getLocations()) {
- result.append(" ");
- result.append(host);
- }
- return result.toString();
- }
- }
-
- static class TextSampler implements IndexedSortable {
- private ArrayList<Text> records = new ArrayList<Text>();
-
- public int compare(int i, int j) {
- Text left = records.get(i);
- Text right = records.get(j);
- return left.compareTo(right);
- }
-
- public void swap(int i, int j) {
- Text left = records.get(i);
- Text right = records.get(j);
- records.set(j, left);
- records.set(i, right);
- }
-
- public void addKey(Text key) {
- synchronized (this) {
- records.add(new Text(key));
- }
- }
-
- /**
- * Find the split points for a given sample. The sample keys are sorted
- * and down sampled to find even split points for the partitions. The
- * returned keys should be the start of their respective partitions.
- * @param numPartitions the desired number of partitions
- * @return an array of size numPartitions - 1 that holds the split points
- */
- Text[] createPartitions(int numPartitions) {
- int numRecords = records.size();
- System.out.println("Making " + numPartitions + " from " + numRecords +
- " sampled records");
- if (numPartitions > numRecords) {
- throw new IllegalArgumentException
- ("Requested more partitions than input keys (" + numPartitions +
- " > " + numRecords + ")");
- }
- new QuickSort().sort(this, 0, records.size());
- float stepSize = numRecords / (float) numPartitions;
- Text[] result = new Text[numPartitions-1];
- for(int i=1; i < numPartitions; ++i) {
- result[i-1] = records.get(Math.round(stepSize * i));
- }
- return result;
- }
- }
-
- /**
- * Use the input splits to take samples of the input and generate sample
- * keys. By default reads 100,000 keys from 10 locations in the input, sorts
- * them and picks N-1 keys to generate N equally sized partitions.
- * @param job the job to sample
- * @param partFile where to write the output file to
- * @throws Throwable if something goes wrong
- */
- public static void writePartitionFile(final JobContext job,
- Path partFile) throws Throwable {
- long t1 = System.currentTimeMillis();
- Configuration conf = job.getConfiguration();
- final TeraInputFormat inFormat = new TeraInputFormat();
- final TextSampler sampler = new TextSampler();
- int partitions = job.getNumReduceTasks();
- long sampleSize = conf.getLong(SAMPLE_SIZE, 100000);
- final List<InputSplit> splits = inFormat.getSplits(job);
- long t2 = System.currentTimeMillis();
- System.out.println("Computing input splits took " + (t2 - t1) + "ms");
- int samples = Math.min(conf.getInt(NUM_PARTITIONS, 10), splits.size());
- System.out.println("Sampling " + samples + " splits of " + splits.size());
- final long recordsPerSample = sampleSize / samples;
- final int sampleStep = splits.size() / samples;
- Thread[] samplerReader = new Thread[samples];
- SamplerThreadGroup threadGroup = new SamplerThreadGroup("Sampler Reader Thread Group");
- // take N samples from different parts of the input
- for(int i=0; i < samples; ++i) {
- final int idx = i;
- samplerReader[i] =
- new Thread (threadGroup,"Sampler Reader " + idx) {
- {
- setDaemon(true);
- }
- public void run() {
- long records = 0;
- try {
- TaskAttemptContext context = new TaskAttemptContextImpl(
- job.getConfiguration(), new TaskAttemptID());
- RecordReader<Text, Text> reader =
- inFormat.createRecordReader(splits.get(sampleStep * idx),
- context);
- reader.initialize(splits.get(sampleStep * idx), context);
- while (reader.nextKeyValue()) {
- sampler.addKey(new Text(reader.getCurrentKey()));
- records += 1;
- if (recordsPerSample <= records) {
- break;
- }
- }
- } catch (IOException ie){
- System.err.println("Got an exception while reading splits " +
- ExceptionUtils.getStackTrace(ie));
- throw new RuntimeException(ie);
- } catch (InterruptedException e) {
-
- }
- }
- };
- samplerReader[i].start();
- }
- FileSystem outFs = partFile.getFileSystem(conf);
- DataOutputStream writer = outFs.create(partFile, true, 64*1024, (short) 10,
- outFs.getDefaultBlockSize(partFile));
- for (int i = 0; i < samples; i++) {
- try {
- samplerReader[i].join();
- if(threadGroup.getThrowable() != null){
- throw threadGroup.getThrowable();
- }
- } catch (InterruptedException e) {
- }
- }
- for(Text split : sampler.createPartitions(partitions)) {
- split.write(writer);
- }
- writer.close();
- long t3 = System.currentTimeMillis();
- System.out.println("Computing parititions took " + (t3 - t2) + "ms");
- }
-
- static class SamplerThreadGroup extends ThreadGroup{
-
- private Throwable throwable;
-
- public SamplerThreadGroup(String s) {
- super(s);
- }
-
- @Override
- public void uncaughtException(Thread thread, Throwable throwable) {
- this.throwable = throwable;
- }
-
- public Throwable getThrowable() {
- return this.throwable;
- }
-
- }
-
- static class TeraRecordReader extends RecordReader<Text,Text> {
- private FSDataInputStream in;
- private long offset;
- private long length;
- private static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
- private byte[] buffer = new byte[RECORD_LENGTH];
- private Text key;
- private Text value;
-
- public TeraRecordReader() throws IOException {
- }
-
- public void initialize(InputSplit split, TaskAttemptContext context)
- throws IOException, InterruptedException {
- Path p = ((FileSplit)split).getPath();
- FileSystem fs = p.getFileSystem(context.getConfiguration());
- in = fs.open(p);
- long start = ((FileSplit)split).getStart();
- // find the offset to start at a record boundary
- offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH;
- in.seek(start + offset);
- length = ((FileSplit)split).getLength();
- }
-
- public void close() throws IOException {
- in.close();
- }
-
- public Text getCurrentKey() {
- return key;
- }
-
- public Text getCurrentValue() {
- return value;
- }
-
- public float getProgress() throws IOException {
- return (float) offset / length;
- }
-
- public boolean nextKeyValue() throws IOException {
- if (offset >= length) {
- return false;
- }
- int read = 0;
- while (read < RECORD_LENGTH) {
- long newRead = in.read(buffer, read, RECORD_LENGTH - read);
- if (newRead == -1) {
- if (read == 0) {
- return false;
- } else {
- throw new EOFException("read past eof");
- }
- }
- read += newRead;
- }
- if (key == null) {
- key = new Text();
- }
- if (value == null) {
- value = new Text();
- }
- key.set(buffer, 0, KEY_LENGTH);
- value.set(buffer, KEY_LENGTH, VALUE_LENGTH);
- offset += RECORD_LENGTH;
- return true;
- }
- }
-
- @Override
- public RecordReader<Text, Text>
- createRecordReader(InputSplit split, TaskAttemptContext context)
- throws IOException {
- return new TeraRecordReader();
- }
-
- protected FileSplit makeSplit(Path file, long start, long length,
- String[] hosts) {
- return new TeraFileSplit(file, start, length, hosts);
- }
-
- @Override
- public List<InputSplit> getSplits(JobContext job) throws IOException {
- if (job == lastContext) {
- return lastResult;
- }
- long t1, t2, t3;
- t1 = System.currentTimeMillis();
- lastContext = job;
- lastResult = super.getSplits(job);
- t2 = System.currentTimeMillis();
- System.out.println("Spent " + (t2 - t1) + "ms computing base-splits.");
- if (job.getConfiguration().getBoolean(TeraScheduler.USE, true)) {
- TeraScheduler scheduler = new TeraScheduler(
- lastResult.toArray(new TeraFileSplit[0]), job.getConfiguration());
- lastResult = scheduler.getNewFileSplits();
- t3 = System.currentTimeMillis();
- System.out.println("Spent " + (t3 - t2) + "ms computing TeraScheduler splits.");
- }
- return lastResult;
- }
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
deleted file mode 100644
index 4ff0f06..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InvalidJobConfException;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.security.TokenCache;
-
-/**
- * An output format that writes the key and value appended together.
- */
-public class TeraOutputFormat extends FileOutputFormat<Text,Text> {
- static final String FINAL_SYNC_ATTRIBUTE = "mapreduce.terasort.final.sync";
- private OutputCommitter committer = null;
-
- /**
- * Set the requirement for a final sync before the stream is closed.
- */
- static void setFinalSync(JobContext job, boolean newValue) {
- job.getConfiguration().setBoolean(FINAL_SYNC_ATTRIBUTE, newValue);
- }
-
- /**
- * Does the user want a final sync at close?
- */
- public static boolean getFinalSync(JobContext job) {
- return job.getConfiguration().getBoolean(FINAL_SYNC_ATTRIBUTE, false);
- }
-
- static class TeraRecordWriter extends RecordWriter<Text,Text> {
- private boolean finalSync = false;
- private FSDataOutputStream out;
-
- public TeraRecordWriter(FSDataOutputStream out,
- JobContext job) {
- finalSync = getFinalSync(job);
- this.out = out;
- }
-
- public synchronized void write(Text key,
- Text value) throws IOException {
- out.write(key.getBytes(), 0, key.getLength());
- out.write(value.getBytes(), 0, value.getLength());
- }
-
- public void close(TaskAttemptContext context) throws IOException {
- if (finalSync) {
- out.hsync();
- }
- out.close();
- }
- }
-
- @Override
- public void checkOutputSpecs(JobContext job
- ) throws InvalidJobConfException, IOException {
- // Ensure that the output directory is set
- Path outDir = getOutputPath(job);
- if (outDir == null) {
- throw new InvalidJobConfException("Output directory not set in JobConf.");
- }
-
- // get delegation token for outDir's file system
- TokenCache.obtainTokensForNamenodes(job.getCredentials(),
- new Path[] { outDir }, job.getConfiguration());
- }
-
- public RecordWriter<Text,Text> getRecordWriter(TaskAttemptContext job
- ) throws IOException {
- Path file = getDefaultWorkFile(job, "");
- FileSystem fs = file.getFileSystem(job.getConfiguration());
- FSDataOutputStream fileOut = fs.create(file);
- return new TeraRecordWriter(fileOut, job);
- }
-
- public OutputCommitter getOutputCommitter(TaskAttemptContext context)
- throws IOException {
- if (committer == null) {
- Path output = getOutputPath(context);
- committer = new FileOutputCommitter(output, context);
- }
- return committer;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
deleted file mode 100644
index 82638d8..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
-import org.apache.tez.mapreduce.examples.terasort.TeraInputFormat.TeraFileSplit;
-
-import com.google.common.base.Charsets;
-
-class TeraScheduler {
- static String USE = "mapreduce.terasort.use.terascheduler";
- private static final Log LOG = LogFactory.getLog(TeraScheduler.class);
- private Split[] splits;
- private List<Host> hosts = new ArrayList<Host>();
- private int slotsPerHost;
- private int remainingSplits = 0;
- private FileSplit[] realSplits = null;
-
- static class Split {
- String filename;
- boolean isAssigned = false;
- List<Host> locations = new ArrayList<Host>();
- Split(String filename) {
- this.filename = filename;
- }
- public String toString() {
- StringBuffer result = new StringBuffer();
- result.append(filename);
- result.append(" on ");
- for(Host host: locations) {
- result.append(host.hostname);
- result.append(", ");
- }
- return result.toString();
- }
- }
- static class Host {
- String hostname;
- List<Split> splits = new ArrayList<Split>();
- Host(String hostname) {
- this.hostname = hostname;
- }
- public String toString() {
- StringBuffer result = new StringBuffer();
- result.append(splits.size());
- result.append(" ");
- result.append(hostname);
- return result.toString();
- }
- }
-
- List<String> readFile(String filename) throws IOException {
- List<String> result = new ArrayList<String>(10000);
- BufferedReader in = new BufferedReader(
- new InputStreamReader(new FileInputStream(filename), Charsets.UTF_8));
- String line = in.readLine();
- while (line != null) {
- result.add(line);
- line = in.readLine();
- }
- in.close();
- return result;
- }
-
- public TeraScheduler(String splitFilename,
- String nodeFilename) throws IOException {
- slotsPerHost = 4;
- // get the hosts
- Map<String, Host> hostIds = new HashMap<String,Host>();
- for(String hostName: readFile(nodeFilename)) {
- Host host = new Host(hostName);
- hosts.add(host);
- hostIds.put(hostName, host);
- }
- // read the blocks
- List<String> splitLines = readFile(splitFilename);
- splits = new Split[splitLines.size()];
- remainingSplits = 0;
- for(String line: splitLines) {
- StringTokenizer itr = new StringTokenizer(line);
- Split newSplit = new Split(itr.nextToken());
- splits[remainingSplits++] = newSplit;
- while (itr.hasMoreTokens()) {
- Host host = hostIds.get(itr.nextToken());
- newSplit.locations.add(host);
- host.splits.add(newSplit);
- }
- }
- }
-
- public TeraScheduler(FileSplit[] realSplits,
- Configuration conf) throws IOException {
- this.realSplits = realSplits;
- this.slotsPerHost = conf.getInt(TTConfig.TT_MAP_SLOTS, 4);
- Map<String, Host> hostTable = new HashMap<String, Host>();
- splits = new Split[realSplits.length];
- for(FileSplit realSplit: realSplits) {
- Split split = new Split(realSplit.getPath().toString());
- splits[remainingSplits++] = split;
- for(String hostname: realSplit.getLocations()) {
- Host host = hostTable.get(hostname);
- if (host == null) {
- host = new Host(hostname);
- hostTable.put(hostname, host);
- hosts.add(host);
- }
- host.splits.add(split);
- split.locations.add(host);
- }
- }
- }
-
- Host pickBestHost() {
- Host result = null;
- int splits = Integer.MAX_VALUE;
- for(Host host: hosts) {
- if (host.splits.size() < splits) {
- result = host;
- splits = host.splits.size();
- }
- }
- if (result != null) {
- hosts.remove(result);
- LOG.debug("picking " + result);
- }
- return result;
- }
-
- void pickBestSplits(Host host) {
- int tasksToPick = Math.min(slotsPerHost,
- (int) Math.ceil((double) remainingSplits /
- hosts.size()));
- Split[] best = new Split[tasksToPick];
- for(Split cur: host.splits) {
- LOG.debug(" examine: " + cur.filename + " " + cur.locations.size());
- int i = 0;
- while (i < tasksToPick && best[i] != null &&
- best[i].locations.size() <= cur.locations.size()) {
- i += 1;
- }
- if (i < tasksToPick) {
- for(int j = tasksToPick - 1; j > i; --j) {
- best[j] = best[j-1];
- }
- best[i] = cur;
- }
- }
- // for the chosen blocks, remove them from the other locations
- for(int i=0; i < tasksToPick; ++i) {
- if (best[i] != null) {
- LOG.debug(" best: " + best[i].filename);
- for (Host other: best[i].locations) {
- other.splits.remove(best[i]);
- }
- best[i].locations.clear();
- best[i].locations.add(host);
- best[i].isAssigned = true;
- remainingSplits -= 1;
- }
- }
- // for the non-chosen blocks, remove this host
- for(Split cur: host.splits) {
- if (!cur.isAssigned) {
- cur.locations.remove(host);
- }
- }
- }
-
- void solve() throws IOException {
- Host host = pickBestHost();
- while (host != null) {
- pickBestSplits(host);
- host = pickBestHost();
- }
- }
-
- /**
- * Solve the schedule and modify the FileSplit array to reflect the new
- * schedule. It will move placed splits to front and unplacable splits
- * to the end.
- * @return a new list of FileSplits that are modified to have the
- * best host as the only host.
- * @throws IOException
- */
- public List<InputSplit> getNewFileSplits() throws IOException {
- solve();
- FileSplit[] result = new FileSplit[realSplits.length];
- int left = 0;
- int right = realSplits.length - 1;
- for(int i=0; i < splits.length; ++i) {
- if (splits[i].isAssigned) {
- // copy the split and fix up the locations
- ((TeraFileSplit) realSplits[i]).setLocations
- (new String[]{splits[i].locations.get(0).hostname});
- result[left++] = realSplits[i];
- } else {
- result[right--] = realSplits[i];
- }
- }
- List<InputSplit> ret = new ArrayList<InputSplit>();
- for (FileSplit fs : result) {
- ret.add(fs);
- }
- return ret;
- }
-
- public static void main(String[] args) throws IOException {
- TeraScheduler problem = new TeraScheduler("block-loc.txt", "nodes");
- for(Host host: problem.hosts) {
- System.out.println(host);
- }
- LOG.info("starting solve");
- problem.solve();
- List<Split> leftOvers = new ArrayList<Split>();
- for(int i=0; i < problem.splits.length; ++i) {
- if (problem.splits[i].isAssigned) {
- System.out.println("sched: " + problem.splits[i]);
- } else {
- leftOvers.add(problem.splits[i]);
- }
- }
- for(Split cur: leftOvers) {
- System.out.println("left: " + cur);
- }
- System.out.println("left over: " + leftOvers.size());
- LOG.info("done");
- }
-
-}
[10/25] git commit: Fix accidental change to CHANGES.txt
Posted by je...@apache.org.
Fix accidental change to CHANGES.txt
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/e709d62a
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/e709d62a
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/e709d62a
Branch: refs/heads/TEZ-8
Commit: e709d62aaf56115a771a361bc83d9d4bdd7bb924
Parents: b4580a7
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 12 03:16:06 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 12 03:16:06 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 -
1 file changed, 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/e709d62a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index dd20f43..4abc9bf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,7 +17,6 @@ Release 0.5.1: Unreleased
INCOMPATIBLE CHANGES
TEZ-1539. Change InputInitializerEvent semantics to SEND_ONCE_ON_TASK_SUCCESS
- TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
ALL CHANGES
TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput()
[13/25] TEZ-1578. Remove TeraSort from Tez codebase. (hitesh)
Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
deleted file mode 100644
index 5a097f2..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
+++ /dev/null
@@ -1,335 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URI;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.Partitioner;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generates the sampled split points, launches the job, and waits for it to
- * finish.
- * <p>
- * To run the program:
- * <b>bin/hadoop jar hadoop-*-examples.jar terasort in-dir out-dir</b>
- */
-public class TeraSort extends Configured implements Tool {
- private static final Log LOG = LogFactory.getLog(TeraSort.class);
- static String SIMPLE_PARTITIONER = "mapreduce.terasort.simplepartitioner";
- static String OUTPUT_REPLICATION = "mapreduce.terasort.output.replication";
-
- /**
- * A partitioner that splits text keys into roughly equal partitions
- * in a global sorted order.
- */
- static class TotalOrderPartitioner extends Partitioner<Text,Text>
- implements Configurable {
- private TrieNode trie;
- private Text[] splitPoints;
- private Configuration conf;
-
- /**
- * A generic trie node
- */
- static abstract class TrieNode {
- private int level;
- TrieNode(int level) {
- this.level = level;
- }
- abstract int findPartition(Text key);
- abstract void print(PrintStream strm) throws IOException;
- int getLevel() {
- return level;
- }
- }
-
- /**
- * An inner trie node that contains 256 children based on the next
- * character.
- */
- static class InnerTrieNode extends TrieNode {
- private TrieNode[] child = new TrieNode[256];
-
- InnerTrieNode(int level) {
- super(level);
- }
- int findPartition(Text key) {
- int level = getLevel();
- if (key.getLength() <= level) {
- return child[0].findPartition(key);
- }
- return child[key.getBytes()[level] & 0xff].findPartition(key);
- }
- void setChild(int idx, TrieNode child) {
- this.child[idx] = child;
- }
- void print(PrintStream strm) throws IOException {
- for(int ch=0; ch < 256; ++ch) {
- for(int i = 0; i < 2*getLevel(); ++i) {
- strm.print(' ');
- }
- strm.print(ch);
- strm.println(" ->");
- if (child[ch] != null) {
- child[ch].print(strm);
- }
- }
- }
- }
-
- /**
- * A leaf trie node that does string compares to figure out where the given
- * key belongs between lower..upper.
- */
- static class LeafTrieNode extends TrieNode {
- int lower;
- int upper;
- Text[] splitPoints;
- LeafTrieNode(int level, Text[] splitPoints, int lower, int upper) {
- super(level);
- this.splitPoints = splitPoints;
- this.lower = lower;
- this.upper = upper;
- }
- int findPartition(Text key) {
- for(int i=lower; i<upper; ++i) {
- if (splitPoints[i].compareTo(key) > 0) {
- return i;
- }
- }
- return upper;
- }
- void print(PrintStream strm) throws IOException {
- for(int i = 0; i < 2*getLevel(); ++i) {
- strm.print(' ');
- }
- strm.print(lower);
- strm.print(", ");
- strm.println(upper);
- }
- }
-
-
- /**
- * Read the cut points from the given sequence file.
- * @param fs the file system
- * @param p the path to read
- * @param job the job config
- * @return the strings to split the partitions on
- * @throws IOException
- */
- private static Text[] readPartitions(FileSystem fs, Path p,
- Configuration conf) throws IOException {
- int reduces = conf.getInt(MRJobConfig.NUM_REDUCES, 1);
- Text[] result = new Text[reduces - 1];
- DataInputStream reader = fs.open(p);
- for(int i=0; i < reduces - 1; ++i) {
- result[i] = new Text();
- result[i].readFields(reader);
- }
- reader.close();
- return result;
- }
-
- /**
- * Given a sorted set of cut points, build a trie that will find the correct
- * partition quickly.
- * @param splits the list of cut points
- * @param lower the lower bound of partitions 0..numPartitions-1
- * @param upper the upper bound of partitions 0..numPartitions-1
- * @param prefix the prefix that we have already checked against
- * @param maxDepth the maximum depth we will build a trie for
- * @return the trie node that will divide the splits correctly
- */
- private static TrieNode buildTrie(Text[] splits, int lower, int upper,
- Text prefix, int maxDepth) {
- int depth = prefix.getLength();
- if (depth >= maxDepth || lower == upper) {
- return new LeafTrieNode(depth, splits, lower, upper);
- }
- InnerTrieNode result = new InnerTrieNode(depth);
- Text trial = new Text(prefix);
- // append an extra byte on to the prefix
- trial.append(new byte[1], 0, 1);
- int currentBound = lower;
- for(int ch = 0; ch < 255; ++ch) {
- trial.getBytes()[depth] = (byte) (ch + 1);
- lower = currentBound;
- while (currentBound < upper) {
- if (splits[currentBound].compareTo(trial) >= 0) {
- break;
- }
- currentBound += 1;
- }
- trial.getBytes()[depth] = (byte) ch;
- result.child[ch] = buildTrie(splits, lower, currentBound, trial,
- maxDepth);
- }
- // pick up the rest
- trial.getBytes()[depth] = (byte) 255;
- result.child[255] = buildTrie(splits, currentBound, upper, trial,
- maxDepth);
- return result;
- }
-
- public void setConf(Configuration conf) {
- try {
- FileSystem fs = FileSystem.getLocal(conf);
- this.conf = conf;
- Path partFile = new Path(TeraInputFormat.PARTITION_FILENAME);
- splitPoints = readPartitions(fs, partFile, conf);
- trie = buildTrie(splitPoints, 0, splitPoints.length, new Text(), 2);
- } catch (IOException ie) {
- throw new IllegalArgumentException("can't read paritions file", ie);
- }
- }
-
- public Configuration getConf() {
- return conf;
- }
-
- public TotalOrderPartitioner() {
- }
-
- public int getPartition(Text key, Text value, int numPartitions) {
- return trie.findPartition(key);
- }
-
- }
-
- /**
- * A total order partitioner that assigns keys based on their first
- * PREFIX_LENGTH bytes, assuming a flat distribution.
- */
- public static class SimplePartitioner extends Partitioner<Text, Text>
- implements Configurable {
- int prefixesPerReduce;
- private static final int PREFIX_LENGTH = 3;
- private Configuration conf = null;
- public void setConf(Configuration conf) {
- this.conf = conf;
- prefixesPerReduce = (int) Math.ceil((1 << (8 * PREFIX_LENGTH)) /
- (float) conf.getInt(MRJobConfig.NUM_REDUCES, 1));
- }
-
- public Configuration getConf() {
- return conf;
- }
-
- @Override
- public int getPartition(Text key, Text value, int numPartitions) {
- byte[] bytes = key.getBytes();
- int len = Math.min(PREFIX_LENGTH, key.getLength());
- int prefix = 0;
- for(int i=0; i < len; ++i) {
- prefix = (prefix << 8) | (0xff & bytes[i]);
- }
- return prefix / prefixesPerReduce;
- }
- }
-
- public static boolean getUseSimplePartitioner(JobContext job) {
- return job.getConfiguration().getBoolean(SIMPLE_PARTITIONER, false);
- }
-
- public static void setUseSimplePartitioner(Job job, boolean value) {
- job.getConfiguration().setBoolean(SIMPLE_PARTITIONER, value);
- }
-
- public static int getOutputReplication(JobContext job) {
- return job.getConfiguration().getInt(OUTPUT_REPLICATION, 1);
- }
-
- public static void setOutputReplication(Job job, int value) {
- job.getConfiguration().setInt(OUTPUT_REPLICATION, value);
- }
-
- public int run(String[] args) throws Exception {
- if (args.length != 2) {
- System.err.println("Invalid no. of arguments provided");
- System.err.println("Usage: terasort <input-dir> <output-dir>");
- return -1;
- }
-
- LOG.info("starting");
- Job job = Job.getInstance(getConf());
- Path inputDir = new Path(args[0]);
- Path outputDir = new Path(args[1]);
- boolean useSimplePartitioner = getUseSimplePartitioner(job);
- TeraInputFormat.setInputPaths(job, inputDir);
- FileOutputFormat.setOutputPath(job, outputDir);
- job.setJobName("TeraSort");
- job.setJarByClass(TeraSort.class);
- job.setOutputKeyClass(Text.class);
- job.setOutputValueClass(Text.class);
- job.setInputFormatClass(TeraInputFormat.class);
- job.setOutputFormatClass(TeraOutputFormat.class);
- if (useSimplePartitioner) {
- job.setPartitionerClass(SimplePartitioner.class);
- } else {
- long start = System.currentTimeMillis();
- Path partitionFile = new Path(outputDir,
- TeraInputFormat.PARTITION_FILENAME);
- URI partitionUri = new URI(partitionFile.toString() +
- "#" + TeraInputFormat.PARTITION_FILENAME);
- try {
- TeraInputFormat.writePartitionFile(job, partitionFile);
- } catch (Throwable e) {
- LOG.error(e.getMessage());
- return -1;
- }
- job.addCacheFile(partitionUri);
- long end = System.currentTimeMillis();
- System.out.println("Spent " + (end - start) + "ms computing partitions.");
- job.setPartitionerClass(TotalOrderPartitioner.class);
- }
-
- job.getConfiguration().setInt("dfs.replication", getOutputReplication(job));
- TeraOutputFormat.setFinalSync(job, true);
- int ret = job.waitForCompletion(true) ? 0 : 1;
- LOG.info("done");
- return ret;
- }
-
- /**
- * @param args
- */
- public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new Configuration(), new TeraSort(), args);
- System.exit(res);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
deleted file mode 100644
index 24c3145..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.IOException;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generate 1 mapper per a file that checks to make sure the keys
- * are sorted within each file. The mapper also generates
- * "$file:begin", first key and "$file:end", last key. The reduce verifies that
- * all of the start/end items are in order.
- * Any output from the reduce is problem report.
- * <p>
- * To run the program:
- * <b>bin/hadoop jar hadoop-*-examples.jar teravalidate out-dir report-dir</b>
- * <p>
- * If there is any output, something is wrong and the output of the reduce
- * will have the problem report.
- */
-public class TeraValidate extends Configured implements Tool {
- private static final Text ERROR = new Text("error");
- private static final Text CHECKSUM = new Text("checksum");
-
- private static String textifyBytes(Text t) {
- BytesWritable b = new BytesWritable();
- b.set(t.getBytes(), 0, t.getLength());
- return b.toString();
- }
-
- static class ValidateMapper extends Mapper<Text,Text,Text,Text> {
- private Text lastKey;
- private String filename;
- private Unsigned16 checksum = new Unsigned16();
- private Unsigned16 tmp = new Unsigned16();
- private Checksum crc32 = new PureJavaCrc32();
-
- /**
- * Get the final part of the input name
- * @param split the input split
- * @return the "part-r-00000" for the input
- */
- private String getFilename(FileSplit split) {
- return split.getPath().getName();
- }
-
- public void map(Text key, Text value, Context context)
- throws IOException, InterruptedException {
- if (lastKey == null) {
- FileSplit fs = (FileSplit) context.getInputSplit();
- filename = getFilename(fs);
- context.write(new Text(filename + ":begin"), key);
- lastKey = new Text();
- } else {
- if (key.compareTo(lastKey) < 0) {
- context.write(ERROR, new Text("misorder in " + filename +
- " between " + textifyBytes(lastKey) +
- " and " + textifyBytes(key)));
- }
- }
- // compute the crc of the key and value and add it to the sum
- crc32.reset();
- crc32.update(key.getBytes(), 0, key.getLength());
- crc32.update(value.getBytes(), 0, value.getLength());
- tmp.set(crc32.getValue());
- checksum.add(tmp);
- lastKey.set(key);
- }
-
- public void cleanup(Context context)
- throws IOException, InterruptedException {
- if (lastKey != null) {
- context.write(new Text(filename + ":end"), lastKey);
- context.write(CHECKSUM, new Text(checksum.toString()));
- }
- }
- }
-
- /**
- * Check the boundaries between the output files by making sure that the
- * boundary keys are always increasing.
- * Also passes any error reports along intact.
- */
- static class ValidateReducer extends Reducer<Text,Text,Text,Text> {
- private boolean firstKey = true;
- private Text lastKey = new Text();
- private Text lastValue = new Text();
- public void reduce(Text key, Iterable<Text> values,
- Context context) throws IOException, InterruptedException {
- if (ERROR.equals(key)) {
- for (Text val : values) {
- context.write(key, val);
- }
- } else if (CHECKSUM.equals(key)) {
- Unsigned16 tmp = new Unsigned16();
- Unsigned16 sum = new Unsigned16();
- for (Text val : values) {
- tmp.set(val.toString());
- sum.add(tmp);
- }
- context.write(CHECKSUM, new Text(sum.toString()));
- } else {
- Text value = values.iterator().next();
- if (firstKey) {
- firstKey = false;
- } else {
- if (value.compareTo(lastValue) < 0) {
- context.write(ERROR,
- new Text("bad key partitioning:\n file " +
- lastKey + " key " +
- textifyBytes(lastValue) +
- "\n file " + key + " key " +
- textifyBytes(value)));
- }
- }
- lastKey.set(key);
- lastValue.set(value);
- }
- }
-
- }
-
- private static void usage() throws IOException {
- System.err.println("teravalidate <out-dir> <report-dir>");
- }
-
- public int run(String[] args) throws Exception {
- Job job = Job.getInstance(getConf());
- if (args.length != 2) {
- usage();
- return 1;
- }
- TeraInputFormat.setInputPaths(job, new Path(args[0]));
- FileOutputFormat.setOutputPath(job, new Path(args[1]));
- job.setJobName("TeraValidate");
- job.setJarByClass(TeraValidate.class);
- job.setMapperClass(ValidateMapper.class);
- job.setReducerClass(ValidateReducer.class);
- job.setOutputKeyClass(Text.class);
- job.setOutputValueClass(Text.class);
- // force a single reducer
- job.setNumReduceTasks(1);
- // force a single split
- FileInputFormat.setMinInputSplitSize(job, Long.MAX_VALUE);
- job.setInputFormatClass(TeraInputFormat.class);
- return job.waitForCompletion(true) ? 0 : 1;
- }
-
- /**
- * @param args
- */
- public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new Configuration(), new TeraValidate(), args);
- System.exit(res);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
deleted file mode 100644
index 1335bf4..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
+++ /dev/null
@@ -1,297 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.examples.terasort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * An unsigned 16 byte integer class that supports addition, multiplication,
- * and left shifts.
- */
-class Unsigned16 implements Writable {
- private long hi8;
- private long lo8;
-
- public Unsigned16() {
- hi8 = 0;
- lo8 = 0;
- }
-
- public Unsigned16(long l) {
- hi8 = 0;
- lo8 = l;
- }
-
- public Unsigned16(Unsigned16 other) {
- hi8 = other.hi8;
- lo8 = other.lo8;
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof Unsigned16) {
- Unsigned16 other = (Unsigned16) o;
- return other.hi8 == hi8 && other.lo8 == lo8;
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return (int) lo8;
- }
-
- /**
- * Parse a hex string
- * @param s the hex string
- */
- public Unsigned16(String s) throws NumberFormatException {
- set(s);
- }
-
- /**
- * Set the number from a hex string
- * @param s the number in hexadecimal
- * @throws NumberFormatException if the number is invalid
- */
- public void set(String s) throws NumberFormatException {
- hi8 = 0;
- lo8 = 0;
- final long lastDigit = 0xfl << 60;
- for (int i = 0; i < s.length(); ++i) {
- int digit = getHexDigit(s.charAt(i));
- if ((lastDigit & hi8) != 0) {
- throw new NumberFormatException(s + " overflowed 16 bytes");
- }
- hi8 <<= 4;
- hi8 |= (lo8 & lastDigit) >>> 60;
- lo8 <<= 4;
- lo8 |= digit;
- }
- }
-
- /**
- * Set the number to a given long.
- * @param l the new value, which is treated as an unsigned number
- */
- public void set(long l) {
- lo8 = l;
- hi8 = 0;
- }
-
- /**
- * Map a hexadecimal character into a digit.
- * @param ch the character
- * @return the digit from 0 to 15
- * @throws NumberFormatException
- */
- private static int getHexDigit(char ch) throws NumberFormatException {
- if (ch >= '0' && ch <= '9') {
- return ch - '0';
- }
- if (ch >= 'a' && ch <= 'f') {
- return ch - 'a' + 10;
- }
- if (ch >= 'A' && ch <= 'F') {
- return ch - 'A' + 10;
- }
- throw new NumberFormatException(ch + " is not a valid hex digit");
- }
-
- private static final Unsigned16 TEN = new Unsigned16(10);
-
- public static Unsigned16 fromDecimal(String s) throws NumberFormatException {
- Unsigned16 result = new Unsigned16();
- Unsigned16 tmp = new Unsigned16();
- for(int i=0; i < s.length(); i++) {
- char ch = s.charAt(i);
- if (ch < '0' || ch > '9') {
- throw new NumberFormatException(ch + " not a valid decimal digit");
- }
- int digit = ch - '0';
- result.multiply(TEN);
- tmp.set(digit);
- result.add(tmp);
- }
- return result;
- }
-
- /**
- * Return the number as a hex string.
- */
- public String toString() {
- if (hi8 == 0) {
- return Long.toHexString(lo8);
- } else {
- StringBuilder result = new StringBuilder();
- result.append(Long.toHexString(hi8));
- String loString = Long.toHexString(lo8);
- for(int i=loString.length(); i < 16; ++i) {
- result.append('0');
- }
- result.append(loString);
- return result.toString();
- }
- }
-
- /**
- * Get a given byte from the number.
- * @param b the byte to get with 0 meaning the most significant byte
- * @return the byte or 0 if b is outside of 0..15
- */
- public byte getByte(int b) {
- if (b >= 0 && b < 16) {
- if (b < 8) {
- return (byte) (hi8 >> (56 - 8*b));
- } else {
- return (byte) (lo8 >> (120 - 8*b));
- }
- }
- return 0;
- }
-
- /**
- * Get the hexadecimal digit at the given position.
- * @param p the digit position to get with 0 meaning the most significant
- * @return the character or '0' if p is outside of 0..31
- */
- public char getHexDigit(int p) {
- byte digit = getByte(p / 2);
- if (p % 2 == 0) {
- digit >>>= 4;
- }
- digit &= 0xf;
- if (digit < 10) {
- return (char) ('0' + digit);
- } else {
- return (char) ('A' + digit - 10);
- }
- }
-
- /**
- * Get the high 8 bytes as a long.
- */
- public long getHigh8() {
- return hi8;
- }
-
- /**
- * Get the low 8 bytes as a long.
- */
- public long getLow8() {
- return lo8;
- }
-
- /**
- * Multiple the current number by a 16 byte unsigned integer. Overflow is not
- * detected and the result is the low 16 bytes of the result. The numbers
- * are divided into 32 and 31 bit chunks so that the product of two chucks
- * fits in the unsigned 63 bits of a long.
- * @param b the other number
- */
- void multiply(Unsigned16 b) {
- // divide the left into 4 32 bit chunks
- long[] left = new long[4];
- left[0] = lo8 & 0xffffffffl;
- left[1] = lo8 >>> 32;
- left[2] = hi8 & 0xffffffffl;
- left[3] = hi8 >>> 32;
- // divide the right into 5 31 bit chunks
- long[] right = new long[5];
- right[0] = b.lo8 & 0x7fffffffl;
- right[1] = (b.lo8 >>> 31) & 0x7fffffffl;
- right[2] = (b.lo8 >>> 62) + ((b.hi8 & 0x1fffffffl) << 2);
- right[3] = (b.hi8 >>> 29) & 0x7fffffffl;
- right[4] = (b.hi8 >>> 60);
- // clear the cur value
- set(0);
- Unsigned16 tmp = new Unsigned16();
- for(int l=0; l < 4; ++l) {
- for (int r=0; r < 5; ++r) {
- long prod = left[l] * right[r];
- if (prod != 0) {
- int off = l*32 + r*31;
- tmp.set(prod);
- tmp.shiftLeft(off);
- add(tmp);
- }
- }
- }
- }
-
- /**
- * Add the given number into the current number.
- * @param b the other number
- */
- public void add(Unsigned16 b) {
- long sumHi;
- long sumLo;
- long reshibit, hibit0, hibit1;
-
- sumHi = hi8 + b.hi8;
-
- hibit0 = (lo8 & 0x8000000000000000L);
- hibit1 = (b.lo8 & 0x8000000000000000L);
- sumLo = lo8 + b.lo8;
- reshibit = (sumLo & 0x8000000000000000L);
- if ((hibit0 & hibit1) != 0 | ((hibit0 ^ hibit1) != 0 && reshibit == 0))
- sumHi++; /* add carry bit */
- hi8 = sumHi;
- lo8 = sumLo;
- }
-
- /**
- * Shift the number a given number of bit positions. The number is the low
- * order bits of the result.
- * @param bits the bit positions to shift by
- */
- public void shiftLeft(int bits) {
- if (bits != 0) {
- if (bits < 64) {
- hi8 <<= bits;
- hi8 |= (lo8 >>> (64 - bits));
- lo8 <<= bits;
- } else if (bits < 128) {
- hi8 = lo8 << (bits - 64);
- lo8 = 0;
- } else {
- hi8 = 0;
- lo8 = 0;
- }
- }
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- hi8 = in.readLong();
- lo8 = in.readLong();
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- out.writeLong(hi8);
- out.writeLong(lo8);
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java b/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java
deleted file mode 100644
index 97b6538..0000000
--- a/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.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.mapreduce.examples.terasort;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.tez.mapreduce.examples.terasort.TeraGen;
-import org.apache.tez.mapreduce.examples.terasort.TeraSort;
-import org.apache.tez.mapreduce.examples.terasort.TeraValidate;
-
-import org.junit.Ignore;
-
-@Ignore
-public class TestTeraSort extends HadoopTestCase {
-
- public TestTeraSort()
- throws IOException {
- super(CLUSTER_MR, DFS_FS, 1, 1);
- }
-
- protected void tearDown() throws Exception {
- getFileSystem().delete(new Path(TEST_DIR), true);
- super.tearDown();
- }
-
- // Input/Output paths for sort
- private static final String TEST_DIR =
- new File(System.getProperty("test.build.data", "/tmp"), "org/apache/tez/mapreduce/examples/terasort")
- .getAbsolutePath();
- private static final Path SORT_INPUT_PATH = new Path(TEST_DIR, "sortin");
- private static final Path SORT_OUTPUT_PATH = new Path(TEST_DIR, "sortout");
- private static final Path TERA_OUTPUT_PATH = new Path(TEST_DIR, "validate");
- private static final String NUM_ROWS = "100";
-
- private void runTeraGen(Configuration conf, Path sortInput)
- throws Exception {
- String[] genArgs = {NUM_ROWS, sortInput.toString()};
-
- // Run TeraGen
- assertEquals(ToolRunner.run(conf, new TeraGen(), genArgs), 0);
- }
-
- private void runTeraSort(Configuration conf,
- Path sortInput, Path sortOutput) throws Exception {
-
- // Setup command-line arguments to 'sort'
- String[] sortArgs = {sortInput.toString(), sortOutput.toString()};
-
- // Run Sort
- assertEquals(ToolRunner.run(conf, new TeraSort(), sortArgs), 0);
- }
-
- private void runTeraValidator(Configuration job,
- Path sortOutput, Path valOutput)
- throws Exception {
- String[] svArgs = {sortOutput.toString(), valOutput.toString()};
-
- // Run Tera-Validator
- assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
- }
-
- public void testTeraSort() throws Exception {
- // Run TeraGen to generate input for 'terasort'
- runTeraGen(createJobConf(), SORT_INPUT_PATH);
-
- // Run terasort
- runTeraSort(createJobConf(), SORT_INPUT_PATH, SORT_OUTPUT_PATH);
-
- // Run tera-validator to check if sort worked correctly
- runTeraValidator(createJobConf(), SORT_OUTPUT_PATH,
- TERA_OUTPUT_PATH);
- }
-
-}
[25/25] git commit: TEZ-1157. Optimize broadcast shuffle to download
data only once per host. (gopalv)
Posted by je...@apache.org.
TEZ-1157. Optimize broadcast shuffle to download data only once per host. (gopalv)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/625450cf
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/625450cf
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/625450cf
Branch: refs/heads/TEZ-8
Commit: 625450cf11454fa9697a902ba70367de00cdc170
Parents: e328055
Author: Gopal V <go...@apache.org>
Authored: Wed Sep 17 20:53:11 2014 -0700
Committer: Gopal V <go...@apache.org>
Committed: Wed Sep 17 20:53:11 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 4 +-
.../library/api/TezRuntimeConfiguration.java | 12 +
.../library/common/InputAttemptIdentifier.java | 26 +-
.../shuffle/common/DiskFetchedInput.java | 7 +
.../library/shuffle/common/FetchedInput.java | 3 +-
.../shuffle/common/FetchedInputAllocator.java | 3 +
.../runtime/library/shuffle/common/Fetcher.java | 318 +++++++++++++++++--
.../impl/ShuffleInputEventHandlerImpl.java | 18 +-
.../shuffle/common/impl/ShuffleManager.java | 42 ++-
.../impl/SimpleFetchedInputAllocator.java | 17 +
10 files changed, 412 insertions(+), 38 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bd5569c..5e2c2cd 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,8 +19,8 @@ ALL CHANGES:
TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
TEZ-1524. Resolve user group information only if ACLs are enabled.
TEZ-1581. GroupByOrderByMRRTest no longer functional.
- TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
- of DAG submission
+ TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless of DAG submission.
+ TEZ-1157. Optimize broadcast shuffle to download data only once per host.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
index 077ce8e..cb61109 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
@@ -280,6 +280,17 @@ public class TezRuntimeConfiguration {
*/
public static final boolean TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH_DEFAULT = false;
+ /**
+ * Share data fetched between tasks running on the same host if applicable
+ */
+ public static final String TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH = TEZ_RUNTIME_PREFIX
+ + "optimize.shared.fetch";
+
+ /**
+ * shared mode bypassing the http fetch is not enabled by default till we have unit tests in.
+ */
+ public static final boolean TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH_DEFAULT = false;
+
// TODO TEZ-1233 - allow this property to be set per vertex
// TODO TEZ-1231 - move these properties out since they are not relevant for Inputs / Outputs
@@ -333,6 +344,7 @@ public class TezRuntimeConfiguration {
tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE);
tezRuntimeKeys.add(TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS);
tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH);
+ tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH);
defaultConf.addResource("core-default.xml");
defaultConf.addResource("core-site.xml");
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/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 7c8a23b..9987d26 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
@@ -29,28 +29,38 @@ public class InputAttemptIdentifier {
private final InputIdentifier inputIdentifier;
private final int attemptNumber;
- private String pathComponent;
-
+ private final String pathComponent;
+ private final boolean shared;
+
public static final String PATH_PREFIX = "attempt";
-
+
public InputAttemptIdentifier(int inputIndex, int attemptNumber) {
this(new InputIdentifier(inputIndex), attemptNumber, null);
}
-
+
public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent) {
+ this(inputIdentifier, attemptNumber, pathComponent, false);
+ }
+
+ public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent, boolean shared) {
this.inputIdentifier = inputIdentifier;
this.attemptNumber = attemptNumber;
this.pathComponent = pathComponent;
+ this.shared = shared;
if (pathComponent != null && !pathComponent.startsWith(PATH_PREFIX)) {
throw new TezUncheckedException(
"Path component must start with: " + PATH_PREFIX + " " + this);
}
}
-
+
public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent) {
this(new InputIdentifier(taskIndex), attemptNumber, pathComponent);
}
+ public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent, boolean shared) {
+ this(new InputIdentifier(taskIndex), attemptNumber, pathComponent, shared);
+ }
+
public InputIdentifier getInputIdentifier() {
return this.inputIdentifier;
}
@@ -63,7 +73,11 @@ public class InputAttemptIdentifier {
return pathComponent;
}
- // PathComponent does not need to be part of the hashCode and equals computation.
+ public boolean isShared() {
+ return this.shared;
+ }
+
+ // PathComponent & shared does not need to be part of the hashCode and equals computation.
@Override
public int hashCode() {
final int prime = 31;
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/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
index 1d26c6e..b0b911b 100644
--- 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
@@ -65,6 +65,13 @@ public class DiskFetchedInput extends FetchedInput {
public InputStream getInputStream() throws IOException {
return localFS.open(outputPath);
}
+
+ public final Path getInputPath() {
+ if (state == State.COMMITTED) {
+ return this.outputPath;
+ }
+ return this.tmpOutputPath;
+ }
@Override
public void commit() throws IOException {
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/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
index fde19b7..0a83dc9 100644
--- 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
@@ -110,7 +110,8 @@ public abstract class FetchedInput {
/**
* Return an input stream to be used to read the previously fetched data.
- * Users are expected to close the InputStream when they're done
+ * All calls to getInputStream() produce new reset streams for reading.
+ * Users are expected to close the InputStream when they're done.
*/
public abstract InputStream getInputStream() throws IOException;
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/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
index 1707ab7..288df6d 100644
--- 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
@@ -21,11 +21,14 @@ package org.apache.tez.runtime.library.shuffle.common;
import java.io.IOException;
import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
public interface FetchedInputAllocator {
public FetchedInput allocate(long actualSize, long compresedSize,
InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
+ public FetchedInput allocateType(Type type, long actualSize, long compresedSize,
+ InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/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
index 9cb8617..e25124b 100644
--- 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
@@ -19,10 +19,17 @@
package org.apache.tez.runtime.library.shuffle.common;
import java.io.DataInputStream;
+import java.io.File;
import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
import java.net.URL;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
import java.util.Arrays;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
@@ -34,12 +41,19 @@ import java.util.concurrent.atomic.AtomicInteger;
import javax.crypto.SecretKey;
import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.io.IOUtils;
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.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -54,6 +68,7 @@ import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
import org.apache.tez.runtime.library.shuffle.common.HttpConnection.HttpConnectionParams;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
/**
* Responsible for fetching inputs served by the ShuffleHandler for a single
@@ -101,10 +116,22 @@ public class Fetcher implements Callable<FetchResult> {
private HttpConnectionParams httpConnectionParams;
private final boolean localDiskFetchEnabled;
+ private final boolean sharedFetchEnabled;
+
+ private final LocalDirAllocator localDirAllocator;
+ private final Path lockPath;
+ private final RawLocalFileSystem localFs;
+
+ private final boolean isDebugEnabled = LOG.isDebugEnabled();
private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
- FetchedInputAllocator inputManager, ApplicationId appId, SecretKey shuffleSecret,
- String srcNameTrimmed, Configuration conf, boolean localDiskFetchEnabled) {
+ FetchedInputAllocator inputManager, ApplicationId appId,
+ SecretKey shuffleSecret, String srcNameTrimmed, Configuration conf,
+ RawLocalFileSystem localFs,
+ LocalDirAllocator localDirAllocator,
+ Path lockPath,
+ boolean localDiskFetchEnabled,
+ boolean sharedFetchEnabled) {
this.fetcherCallback = fetcherCallback;
this.inputManager = inputManager;
this.shuffleSecret = shuffleSecret;
@@ -114,19 +141,42 @@ public class Fetcher implements Callable<FetchResult> {
this.conf = conf;
this.localDiskFetchEnabled = localDiskFetchEnabled;
+ this.sharedFetchEnabled = sharedFetchEnabled;
this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
- this.logIdentifier = "fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier;
+ this.logIdentifier = " fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier;
+
+ this.localFs = localFs;
+ this.localDirAllocator = localDirAllocator;
+ this.lockPath = lockPath;
+
+ try {
+ if (this.sharedFetchEnabled) {
+ this.localFs.mkdirs(this.lockPath);
+ }
+ } catch (Exception e) {
+ LOG.warn("Error initializing local dirs for shared transfer " + e);
+ }
}
@Override
public FetchResult call() throws Exception {
+ boolean multiplex = (this.sharedFetchEnabled && this.localDiskFetchEnabled);
+
if (srcAttempts.size() == 0) {
return new FetchResult(host, port, partition, srcAttempts);
}
for (InputAttemptIdentifier in : srcAttempts) {
pathToAttemptMap.put(in.getPathComponent(), in);
+ // do only if all of them are shared fetches
+ multiplex &= in.isShared();
+ }
+
+ if (multiplex) {
+ Preconditions.checkArgument(partition == 0,
+ "Shared fetches cannot be done for partitioned input"
+ + "- partition is non-zero (%d)", partition);
}
remaining = new LinkedHashSet<InputAttemptIdentifier>(srcAttempts);
@@ -136,7 +186,9 @@ public class Fetcher implements Callable<FetchResult> {
if (localDiskFetchEnabled &&
host.equals(System.getenv(ApplicationConstants.Environment.NM_HOST.toString()))) {
hostFetchResult = setupLocalDiskFetch();
- } else {
+ } else if (multiplex) {
+ hostFetchResult = doSharedFetch();
+ } else{
hostFetchResult = doHttpFetch();
}
@@ -151,15 +203,197 @@ public class Fetcher implements Callable<FetchResult> {
// Sanity check
if (hostFetchResult.failedInputs == null && !remaining.isEmpty()) {
- throw new IOException("server didn't return all expected map outputs: "
- + remaining.size() + " left.");
+ if (!multiplex) {
+ throw new IOException("server didn't return all expected map outputs: "
+ + remaining.size() + " left.");
+ } else {
+ LOG.info("Shared fetch failed to return " + remaining.size() + " inputs on this try");
+ }
}
return hostFetchResult.fetchResult;
}
+ private final class CachingCallBack {
+ // this is a closure object wrapping this in an inner class
+ public void cache(String host,
+ InputAttemptIdentifier srcAttemptId, FetchedInput fetchedInput,
+ long compressedLength, long decompressedLength) {
+ try {
+ // this breaks badly on partitioned input - please use responsibly
+ Preconditions.checkArgument(partition == 0, "Partition == 0");
+ final String tmpSuffix = "." + System.currentTimeMillis() + ".tmp";
+ final String finalOutput = getMapOutputFile(srcAttemptId.getPathComponent());
+ final Path outputPath = localDirAllocator.getLocalPathForWrite(finalOutput, compressedLength, conf);
+ final TezSpillRecord spillRec = new TezSpillRecord(1);
+ final TezIndexRecord indexRec;
+ Path tmpIndex = outputPath.suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING+tmpSuffix);
+
+ if (localFs.exists(tmpIndex)) {
+ LOG.warn("Found duplicate instance of input index file " + tmpIndex);
+ return;
+ }
+
+ Path tmpPath = null;
+
+ switch (fetchedInput.getType()) {
+ case DISK: {
+ DiskFetchedInput input = (DiskFetchedInput) fetchedInput;
+ indexRec = new TezIndexRecord(0, decompressedLength, compressedLength);
+ localFs.mkdirs(outputPath.getParent());
+ // avoid pit-falls of speculation
+ tmpPath = outputPath.suffix(tmpSuffix);
+ // JDK7 - TODO: use Files implementation to speed up this process
+ localFs.copyFromLocalFile(input.getInputPath(), tmpPath);
+ // rename is atomic
+ boolean renamed = localFs.rename(tmpPath, outputPath);
+ if(!renamed) {
+ LOG.warn("Could not rename to cached file name " + outputPath);
+ localFs.delete(tmpPath, false);
+ return;
+ }
+ }
+ break;
+ default:
+ LOG.warn("Incorrect use of CachingCallback for " + srcAttemptId);
+ return;
+ }
+
+ spillRec.putIndex(indexRec, 0);
+ spillRec.writeToFile(tmpIndex, conf);
+ // everything went well so far - rename it
+ boolean renamed = localFs.rename(tmpIndex, outputPath
+ .suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING));
+ if (!renamed) {
+ localFs.delete(tmpIndex, false);
+ if (outputPath != null) {
+ // invariant: outputPath was renamed from tmpPath
+ localFs.delete(outputPath, false);
+ }
+ LOG.warn("Could not rename the index file to "
+ + outputPath
+ .suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING));
+ return;
+ }
+ } catch (IOException ioe) {
+ // do mostly nothing
+ LOG.warn("Cache threw an error " + ioe);
+ }
+ }
+ }
+
+ private int findInputs() throws IOException {
+ int k = 0;
+ for (InputAttemptIdentifier src : srcAttempts) {
+ try {
+ if (getShuffleInputFileName(src.getPathComponent(),
+ Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING) != null) {
+ k++;
+ }
+ } catch (DiskErrorException de) {
+ // missing file, ignore
+ }
+ }
+ return k;
+ }
+
+ private FileLock getLock() throws OverlappingFileLockException, InterruptedException, IOException {
+ File lockFile = localFs.pathToFile(new Path(lockPath, host + ".lock"));
+
+ final boolean created = lockFile.createNewFile();
+
+ if (created == false && !lockFile.exists()) {
+ // bail-out cleanly
+ return null;
+ }
+
+ // invariant - file created (winner writes to this file)
+ // caveat: closing lockChannel does close the file (do not double close)
+ // JDK7 - TODO: use AsynchronousFileChannel instead of RandomAccessFile
+ FileChannel lockChannel = new RandomAccessFile(lockFile, "rws")
+ .getChannel();
+ FileLock xlock = null;
+
+ xlock = lockChannel.tryLock(0, Long.MAX_VALUE, false);
+ if (xlock != null) {
+ return xlock;
+ }
+ lockChannel.close();
+ return null;
+ }
+
+ private void releaseLock(FileLock lock) throws IOException {
+ if (lock != null && lock.isValid()) {
+ FileChannel lockChannel = lock.channel();
+ lock.release();
+ lockChannel.close();
+ }
+ }
+
+ protected HostFetchResult doSharedFetch() throws IOException {
+ int inputs = findInputs();
+
+ if (inputs == srcAttempts.size()) {
+ if (isDebugEnabled) {
+ LOG.debug("Using the copies found locally");
+ }
+ return doLocalDiskFetch(true);
+ }
+
+ if (inputs > 0) {
+ if (isDebugEnabled) {
+ LOG.debug("Found " + input
+ + " local fetches right now, using them first");
+ }
+ return doLocalDiskFetch(false);
+ }
+
+ FileLock lock = null;
+ try {
+ lock = getLock();
+ if (lock == null) {
+ // re-queue until we get a lock
+ LOG.info("Requeuing " + host + ":" + port
+ + " downloads because we didn't get a lock");
+ return new HostFetchResult(new FetchResult(host, port, partition,
+ remaining), null, false);
+ } else {
+ if (findInputs() == srcAttempts.size()) {
+ // double checked after lock
+ releaseLock(lock);
+ lock = null;
+ return doLocalDiskFetch(true);
+ }
+ // cache data if possible
+ return doHttpFetch(new CachingCallBack());
+ }
+ } catch (OverlappingFileLockException jvmCrossLock) {
+ // fall back to HTTP fetch below
+ LOG.warn("Double locking detected for " + host);
+ } catch (InterruptedException sleepInterrupted) {
+ // fall back to HTTP fetch below
+ LOG.warn("Lock was interrupted for " + host);
+ } finally {
+ releaseLock(lock);
+ }
+
+ if (isShutDown.get()) {
+ // if any exception was due to shut-down don't bother firing any more
+ // requests
+ return new HostFetchResult(new FetchResult(host, port, partition,
+ remaining), null, false);
+ }
+ // no more caching
+ return doHttpFetch();
+ }
+
@VisibleForTesting
protected HostFetchResult doHttpFetch() {
+ return doHttpFetch(null);
+ }
+
+ @VisibleForTesting
+ protected HostFetchResult doHttpFetch(CachingCallBack callback) {
try {
StringBuilder baseURI = ShuffleUtils.constructBaseURIForShuffleHandler(host,
port, partition, appId.toString(), httpConnectionParams.isSSLShuffleEnabled());
@@ -227,7 +461,7 @@ public class Fetcher implements Callable<FetchResult> {
// yet_to_be_fetched list and marking the failed tasks.
InputAttemptIdentifier[] failedInputs = null;
while (!remaining.isEmpty() && failedInputs == null) {
- failedInputs = fetchInputs(input);
+ failedInputs = fetchInputs(input, callback);
}
return new HostFetchResult(new FetchResult(host, port, partition, remaining), failedInputs,
@@ -236,6 +470,11 @@ public class Fetcher implements Callable<FetchResult> {
@VisibleForTesting
protected HostFetchResult setupLocalDiskFetch() {
+ return doLocalDiskFetch(true);
+ }
+
+ @VisibleForTesting
+ private HostFetchResult doLocalDiskFetch(boolean failMissing) {
Iterator<InputAttemptIdentifier> iterator = remaining.iterator();
while (iterator.hasNext()) {
@@ -246,6 +485,7 @@ public class Fetcher implements Callable<FetchResult> {
FetchedInput fetchedInput = null;
try {
TezIndexRecord idxRecord;
+ // for missing files, this will throw an exception
idxRecord = getTezIndexRecord(srcAttemptId);
fetchedInput = new LocalDiskFetchedInput(idxRecord.getStartOffset(),
@@ -283,8 +523,10 @@ public class Fetcher implements Callable<FetchResult> {
}
InputAttemptIdentifier[] failedFetches = null;
- if (remaining.size() > 0) {
+ if (failMissing && remaining.size() > 0) {
failedFetches = remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+ } else {
+ // nothing needs to be done to requeue remaining entries
}
return new HostFetchResult(new FetchResult(host, port, partition, remaining),
failedFetches, false);
@@ -296,19 +538,24 @@ public class Fetcher implements Callable<FetchResult> {
TezIndexRecord idxRecord;
Path indexFile = getShuffleInputFileName(srcAttemptId.getPathComponent(),
Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+
TezSpillRecord spillRecord = new TezSpillRecord(indexFile, conf);
idxRecord = spillRecord.getIndex(partition);
return idxRecord;
}
+ private static final String getMapOutputFile(String pathComponent) {
+ return Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+ + pathComponent + Path.SEPARATOR
+ + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING;
+ }
+
@VisibleForTesting
- protected Path getShuffleInputFileName(String pathComponent, String suffix) throws IOException {
- LocalDirAllocator localDirAllocator = new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS);
+ protected Path getShuffleInputFileName(String pathComponent, String suffix)
+ throws IOException {
suffix = suffix != null ? suffix : "";
- String pathFromLocalDir = Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR + pathComponent +
- Path.SEPARATOR + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + suffix;
-
+ String pathFromLocalDir = getMapOutputFile(pathComponent) + suffix;
return localDirAllocator.getLocalPathToRead(pathFromLocalDir, conf);
}
@@ -350,7 +597,7 @@ public class Fetcher implements Callable<FetchResult> {
}
}
- private InputAttemptIdentifier[] fetchInputs(DataInputStream input) {
+ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, CachingCallBack callback) {
FetchedInput fetchedInput = null;
InputAttemptIdentifier srcAttemptId = null;
long decompressedLength = -1;
@@ -392,12 +639,16 @@ public class Fetcher implements Callable<FetchResult> {
LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength
+ ", decomp len: " + decompressedLength);
}
-
- // Get the location for the map output - either in-memory or on-disk
// TODO TEZ-957. handle IOException here when Broadcast has better error checking
- fetchedInput = inputManager.allocate(decompressedLength, compressedLength, srcAttemptId);
-
+ if (srcAttemptId.isShared() && callback != null) {
+ // force disk if input is being shared
+ fetchedInput = inputManager.allocateType(Type.DISK, decompressedLength,
+ compressedLength, srcAttemptId);
+ } else {
+ fetchedInput = inputManager.allocate(decompressedLength,
+ compressedLength, srcAttemptId);
+ }
// TODO NEWTEZ No concept of WAIT at the moment.
// // Check if we can shuffle *now* ...
// if (fetchedInput.getType() == FetchedInput.WAIT) {
@@ -427,6 +678,14 @@ public class Fetcher implements Callable<FetchResult> {
fetchedInput);
}
+ // offer the fetched input for caching
+ if (srcAttemptId.isShared() && callback != null) {
+ // this has to be before the fetchSucceeded, because that goes across
+ // threads into the reader thread and can potentially shutdown this thread
+ // while it is still caching.
+ callback.cache(host, srcAttemptId, fetchedInput, compressedLength, decompressedLength);
+ }
+
// Inform the shuffle scheduler
long endTime = System.currentTimeMillis();
fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput,
@@ -434,6 +693,7 @@ public class Fetcher implements Callable<FetchResult> {
// Note successful shuffle
remaining.remove(srcAttemptId);
+
// metrics.successFetch();
return null;
} catch (IOException ioe) {
@@ -521,12 +781,24 @@ public class Fetcher implements Callable<FetchResult> {
private Fetcher fetcher;
private boolean workAssigned = false;
- public FetcherBuilder(FetcherCallback fetcherCallback, HttpConnectionParams params,
- FetchedInputAllocator inputManager, ApplicationId appId,
- SecretKey shuffleSecret, String srcNameTrimmed, Configuration conf,
- boolean localDiskFetchEnabled) {
+ public FetcherBuilder(FetcherCallback fetcherCallback,
+ HttpConnectionParams params, FetchedInputAllocator inputManager,
+ ApplicationId appId, SecretKey shuffleSecret, String srcNameTrimmed,
+ Configuration conf, boolean localDiskFetchEnabled) {
+ this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
+ shuffleSecret, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled,
+ false);
+ }
+
+ public FetcherBuilder(FetcherCallback fetcherCallback,
+ HttpConnectionParams params, FetchedInputAllocator inputManager,
+ ApplicationId appId, SecretKey shuffleSecret, String srcNameTrimmed,
+ Configuration conf, RawLocalFileSystem localFs,
+ LocalDirAllocator localDirAllocator, Path lockPath,
+ boolean localDiskFetchEnabled, boolean sharedFetchEnabled) {
this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
- shuffleSecret, srcNameTrimmed, conf, localDiskFetchEnabled);
+ shuffleSecret, srcNameTrimmed, conf, localFs, localDirAllocator,
+ lockPath, localDiskFetchEnabled, sharedFetchEnabled);
}
public FetcherBuilder setHttpConnectionParameters(HttpConnectionParams httpParams) {
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/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 9d621e8..2ac45d4 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
@@ -24,6 +24,7 @@ import java.util.BitSet;
import java.util.List;
import com.google.protobuf.ByteString;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.compress.CompressionCodec;
@@ -53,6 +54,7 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
private final CompressionCodec codec;
private final boolean ifileReadAhead;
private final int ifileReadAheadLength;
+ private final boolean useSharedInputs;
public ShuffleInputEventHandlerImpl(InputContext inputContext,
ShuffleManager shuffleManager,
@@ -63,6 +65,9 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
this.codec = codec;
this.ifileReadAhead = ifileReadAhead;
this.ifileReadAheadLength = ifileReadAheadLength;
+ // this currently relies on a user to enable the flag
+ // expand on idea based on vertex parallelism and num inputs
+ this.useSharedInputs = (inputContext.getTaskAttemptNumber() == 0);
}
@Override
@@ -109,9 +114,12 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier);
return;
}
- }
- InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(),
- dme.getVersion(), shufflePayload.getPathComponent());
+ }
+
+ InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(
+ dme.getTargetIndex(), dme.getVersion(),
+ shufflePayload.getPathComponent(), (useSharedInputs && srcIndex == 0));
+
if (shufflePayload.hasData()) {
DataProto dataProto = shufflePayload.getData();
FetchedInput fetchedInput = inputAllocator.allocate(dataProto.getRawLength(),
@@ -119,8 +127,8 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
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);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
index 4e1a06c..8aa6582 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.text.DecimalFormat;
+import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
@@ -43,7 +44,12 @@ import javax.crypto.SecretKey;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.tez.common.TezRuntimeFrameworkConfigs;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.common.counters.TaskCounter;
import org.apache.tez.common.counters.TezCounter;
@@ -68,7 +74,9 @@ import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
import org.apache.tez.runtime.library.shuffle.common.Fetcher.FetcherBuilder;
+import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
@@ -120,6 +128,7 @@ public class ShuffleManager implements FetcherCallback {
private final SecretKey shuffleSecret;
private final CompressionCodec codec;
private final boolean localDiskFetchEnabled;
+ private final boolean sharedFetchEnabled;
private final int ifileBufferSize;
private final boolean ifileReadAhead;
@@ -140,6 +149,11 @@ public class ShuffleManager implements FetcherCallback {
private volatile Throwable shuffleError;
private final HttpConnectionParams httpConnectionParams;
+
+ private final LocalDirAllocator localDirAllocator;
+ private final RawLocalFileSystem localFs;
+ private final Path[] localDisks;
+
// TODO More counters - FetchErrors, speed?
public ShuffleManager(InputContext inputContext, Configuration conf, int numInputs,
@@ -163,6 +177,8 @@ public class ShuffleManager implements FetcherCallback {
this.inputManager = inputAllocator;
this.localDiskFetchEnabled = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH,
TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH_DEFAULT);
+ this.sharedFetchEnabled = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH,
+ TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH_DEFAULT);
this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName());
@@ -199,10 +215,23 @@ public class ShuffleManager implements FetcherCallback {
.getServiceConsumerMetaData(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID));
httpConnectionParams =
ShuffleUtils.constructHttpShuffleConnectionParams(conf);
+
+ this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw();
+
+ this.localDirAllocator = new LocalDirAllocator(
+ TezRuntimeFrameworkConfigs.LOCAL_DIRS);
+
+ this.localDisks = Iterables.toArray(
+ localDirAllocator.getAllLocalPathsToRead(".", conf), Path.class);
+
+ Arrays.sort(this.localDisks);
+
LOG.info(this.getClass().getSimpleName() + " : numInputs=" + numInputs + ", compressionCodec="
+ (codec == null ? "NoCompressionCodec" : codec.getClass().getName()) + ", numFetchers="
+ numFetchers + ", ifileBufferSize=" + ifileBufferSize + ", ifileReadAheadEnabled="
+ ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", "
+ + "localDiskFetchEnabled=" + localDiskFetchEnabled + ", "
+ + "sharedFetchEnabled=" + sharedFetchEnabled + ", "
+ httpConnectionParams.toString());
}
@@ -301,9 +330,20 @@ public class ShuffleManager implements FetcherCallback {
}
private Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) {
+
+ Path lockDisk = null;
+
+ if (sharedFetchEnabled) {
+ // pick a single lock disk from the edge name's hashcode + host hashcode
+ final int h = Math.abs(Objects.hashCode(this.srcNameTrimmed, inputHost.getHost()));
+ lockDisk = new Path(this.localDisks[h % this.localDisks.length], "locks");
+ }
+
FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this,
httpConnectionParams, inputManager, inputContext.getApplicationId(),
- shuffleSecret, srcNameTrimmed, conf, localDiskFetchEnabled);
+ shuffleSecret, srcNameTrimmed, conf, localFs, localDirAllocator,
+ lockDisk, localDiskFetchEnabled, sharedFetchEnabled);
+
if (codec != null) {
fetcherBuilder.setCompressionParameters(codec);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
index d365aa4..20ee665 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
@@ -33,6 +33,7 @@ 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.FetchedInput.Type;
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;
@@ -140,10 +141,26 @@ public class SimpleFetchedInputAllocator implements FetchedInputAllocator,
}
@Override
+ public synchronized FetchedInput allocateType(Type type, long actualSize,
+ long compressedSize, InputAttemptIdentifier inputAttemptIdentifier)
+ throws IOException {
+
+ switch (type) {
+ case DISK:
+ return new DiskFetchedInput(actualSize, compressedSize,
+ inputAttemptIdentifier, this, conf, localDirAllocator,
+ fileNameAllocator);
+ default:
+ return allocate(actualSize, compressedSize, inputAttemptIdentifier);
+ }
+ }
+
+ @Override
public synchronized void fetchComplete(FetchedInput fetchedInput) {
switch (fetchedInput.getType()) {
// Not tracking anything here.
case DISK:
+ case DISK_DIRECT:
case MEMORY:
break;
default:
[24/25] git commit: TEZ-1587. Some tez-examples fail in local mode
(Contributed by Prakash Ramachandran)
Posted by je...@apache.org.
TEZ-1587. Some tez-examples fail in local mode (Contributed by Prakash Ramachandran)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/e328055e
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/e328055e
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/e328055e
Branch: refs/heads/TEZ-8
Commit: e328055e61a8dcaa1beed25e6da47f4a2279f3b6
Parents: 8bd2ca3
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Wed Sep 17 15:54:01 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Wed Sep 17 15:54:01 2014 +0530
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../java/org/apache/tez/examples/JoinExample.java | 4 ++--
.../org/apache/tez/examples/JoinValidate.java | 2 +-
.../org/apache/tez/examples/OrderedWordCount.java | 4 ++--
.../java/org/apache/tez/examples/WordCount.java | 2 +-
.../examples/BroadcastAndOneToOneExample.java | 18 ++++++++++++++----
.../tez/mapreduce/examples/FilterLinesByWord.java | 3 ++-
.../examples/FilterLinesByWordOneToOne.java | 3 ++-
8 files changed, 25 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 83048bb..bd5569c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -43,6 +43,7 @@ ALL CHANGES
TEZ-1571. Add create method for DataSinkDescriptor.
TEZ-1585. Memory leak in tez session mode.
TEZ-1533. Request Events more often if a complete set of events is received by a task.
+ TEZ-1587. Some tez-examples fail in local mode.
Release 0.5.0: 2014-09-03
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
index 5ead275..3611fd6 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
@@ -244,7 +244,7 @@ public class JoinExample extends Configured implements Tool {
UnorderedPartitionedKVEdgeConfig streamConf =
UnorderedPartitionedKVEdgeConfig
.newBuilder(Text.class.getName(), NullWritable.class.getName(),
- HashPartitioner.class.getName()).build();
+ HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
/**
* Connect the join vertex with the stream side
@@ -265,7 +265,7 @@ public class JoinExample extends Configured implements Tool {
* value is null.
*/
UnorderedKVEdgeConfig broadcastConf = UnorderedKVEdgeConfig.newBuilder(Text.class.getName(),
- NullWritable.class.getName()).build();
+ NullWritable.class.getName()).setFromConfiguration(tezConf).build();
hashSideEdgeProperty = broadcastConf.createDefaultBroadcastEdgeProperty();
} else {
/**
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
index 3630184..e94df09 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
@@ -186,7 +186,7 @@ public class JoinValidate extends Configured implements Tool {
// better mechanism to configure the IOs.
OrderedPartitionedKVEdgeConfig edgeConf = OrderedPartitionedKVEdgeConfig
.newBuilder(Text.class.getName(), NullWritable.class.getName(),
- HashPartitioner.class.getName()).build();
+ HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
Vertex lhsVertex = Vertex.create(LHS_INPUT_NAME, ProcessorDescriptor.create(
ForwardingProcessor.class.getName())).addDataSource("lhs",
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java b/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
index fd18592..48884f4 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
@@ -140,7 +140,7 @@ public class OrderedWordCount extends Configured implements Tool {
// Use Text key and IntWritable value to bring counts for each word in the same partition
OrderedPartitionedKVEdgeConfig summationEdgeConf = OrderedPartitionedKVEdgeConfig
.newBuilder(Text.class.getName(), IntWritable.class.getName(),
- HashPartitioner.class.getName()).build();
+ HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
// This vertex will be reading intermediate data via an input edge and writing intermediate data
// via an output edge.
@@ -151,7 +151,7 @@ public class OrderedWordCount extends Configured implements Tool {
// partition. The data will be ordered by count and words grouped by count.
OrderedPartitionedKVEdgeConfig sorterEdgeConf = OrderedPartitionedKVEdgeConfig
.newBuilder(IntWritable.class.getName(), Text.class.getName(),
- HashPartitioner.class.getName()).build();
+ HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
// Use 1 task to bring all the data in one place for global sorted order. Essentially the number
// of partitions is 1. So the NoOpSorter can be used to produce the globally ordered output
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java b/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
index e0e94cb..89eb68c 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
@@ -173,7 +173,7 @@ public class WordCount extends Configured implements Tool {
// using this helper.
OrderedPartitionedKVEdgeConfig edgeConf = OrderedPartitionedKVEdgeConfig
.newBuilder(Text.class.getName(), IntWritable.class.getName(),
- HashPartitioner.class.getName()).build();
+ HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
// Create a vertex that reads the tokenized data and calculates the sum using the SumProcessor.
// The number of tasks that do the work of this vertex depends on the number of partitions used
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
index 81f9581..e8a137c 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
@@ -50,6 +50,7 @@ import org.apache.tez.runtime.api.ObjectRegistry;
import org.apache.tez.runtime.api.ProcessorContext;
import org.apache.tez.runtime.library.api.KeyValueReader;
import org.apache.tez.runtime.library.api.KeyValueWriter;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
import org.apache.tez.runtime.library.conf.UnorderedKVEdgeConfig;
import org.apache.tez.runtime.library.output.UnorderedKVOutput;
import org.apache.tez.runtime.library.processor.SimpleProcessor;
@@ -159,7 +160,8 @@ public class BroadcastAndOneToOneExample extends Configured implements Tool {
VertexManagerPluginDescriptor.create(InputReadyVertexManager.class.getName()));
UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
- .newBuilder(Text.class.getName(), IntWritable.class.getName()).build();
+ .newBuilder(Text.class.getName(), IntWritable.class.getName())
+ .setFromConfiguration(tezConf).build();
DAG dag = DAG.create("BroadcastAndOneToOneExample");
dag.addVertex(inputVertex)
@@ -188,9 +190,9 @@ public class BroadcastAndOneToOneExample extends Configured implements Tool {
// staging dir
FileSystem fs = FileSystem.get(tezConf);
- String stagingDirStr = Path.SEPARATOR + "user" + Path.SEPARATOR
- + user + Path.SEPARATOR+ ".staging" + Path.SEPARATOR
- + Path.SEPARATOR + Long.toString(System.currentTimeMillis());
+ String stagingDirStr = tezConf.get(TezConfiguration.TEZ_AM_STAGING_DIR,
+ TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT) + Path.SEPARATOR +
+ "BroadcastAndOneToOneExample" + Path.SEPARATOR + Long.toString(System.currentTimeMillis());
Path stagingDir = new Path(stagingDirStr);
tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirStr);
stagingDir = fs.makeQualified(stagingDir);
@@ -240,6 +242,14 @@ public class BroadcastAndOneToOneExample extends Configured implements Tool {
printUsage();
throw new TezException("Invalid command line");
}
+
+ if (doLocalityCheck &&
+ getConf().getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
+ TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
+ System.out.println("locality check is not valid in local mode. skipping");
+ doLocalityCheck = false;
+ }
+
boolean status = run(getConf(), doLocalityCheck);
return status ? 0 : 1;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
index 8430c68..f45b3d6 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
@@ -199,7 +199,8 @@ public class FilterLinesByWord extends Configured implements Tool {
stage2Vertex.addDataSink("MROutput", DataSinkDescriptor.create(od, ocd, null));
UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
- .newBuilder(Text.class.getName(), TextLongPair.class.getName()).build();
+ .newBuilder(Text.class.getName(), TextLongPair.class.getName())
+ .setFromConfiguration(tezConf).build();
DAG dag = DAG.create("FilterLinesByWord");
Edge edge =
http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
index 515cf70..49cb30b 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
@@ -190,7 +190,8 @@ public class FilterLinesByWordOneToOne extends Configured implements Tool {
OutputCommitterDescriptor.create(MROutputCommitter.class.getName()), null));
UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
- .newBuilder(Text.class.getName(), TextLongPair.class.getName()).build();
+ .newBuilder(Text.class.getName(), TextLongPair.class.getName())
+ .setFromConfiguration(tezConf).build();
DAG dag = DAG.create("FilterLinesByWord");
Edge edge =
[11/25] git commit: TEZ-1574. Support additional formats for the tez
deployed archive. (sseth)
Posted by je...@apache.org.
TEZ-1574. Support additional formats for the tez deployed archive.
(sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/d1ea44d0
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/d1ea44d0
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/d1ea44d0
Branch: refs/heads/TEZ-8
Commit: d1ea44d0b4f53f729fc7842b5325d0f941b6dd1c
Parents: e709d62
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 12 03:17:49 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 12 03:17:49 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../src/main/java/org/apache/tez/client/TezClientUtils.java | 6 +++++-
2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/d1ea44d0/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4abc9bf..ff53e04 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -28,6 +28,7 @@ ALL CHANGES
TEZ-1362. Remove DAG_COMPLETED in DAGEventType.
TEZ-1519. TezTaskRunner should not initialize TezConfiguration in TezChild.
TEZ-1534. Make client side configs available to AM and tasks.
+ TEZ-1574. Support additional formats for the tez deployed archive
Release 0.5.0: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/d1ea44d0/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index adda0a9..3297adc 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -171,7 +171,11 @@ public class TezClientUtils {
List<Path> tezJarPaths = Lists.newArrayListWithCapacity(tezJarUris.length);
- if (tezJarUris.length == 1 && (tezJarUris[0].endsWith(".tar.gz") || tezJarUris[0].endsWith(".tgz"))) {
+ if (tezJarUris.length == 1 && (
+ tezJarUris[0].endsWith(".tar.gz") ||
+ tezJarUris[0].endsWith(".tgz") ||
+ tezJarUris[0].endsWith(".zip") ||
+ tezJarUris[0].endsWith(".tar"))) {
String fileName = tezJarUris[0];
if (fileName.endsWith(".tar.gz") || fileName.endsWith(".tgz")) {
FileStatus fStatus = getLRFileStatus(fileName, conf, false)[0];
[15/25] git commit: TEZ-1569. Add tests for preemption (bikas)
Posted by je...@apache.org.
TEZ-1569. Add tests for preemption (bikas)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/5e5683ab
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/5e5683ab
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/5e5683ab
Branch: refs/heads/TEZ-8
Commit: 5e5683ab5e89f1fd88883ed40574c8fb04316dd1
Parents: 8e382b3
Author: Bikas Saha <bi...@apache.org>
Authored: Fri Sep 12 14:15:56 2014 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Fri Sep 12 14:15:56 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 3 +-
.../java/org/apache/tez/client/TezClient.java | 2 +-
.../java/org/apache/tez/client/LocalClient.java | 15 +-
.../org/apache/tez/dag/app/DAGAppMaster.java | 15 +-
.../apache/tez/dag/app/dag/impl/DAGImpl.java | 6 +
.../apache/tez/dag/app/MockDAGAppMaster.java | 245 +++++++++++++++++++
.../org/apache/tez/dag/app/MockLocalClient.java | 48 ++++
.../org/apache/tez/dag/app/MockTezClient.java | 48 ++++
.../org/apache/tez/dag/app/TestPreemption.java | 203 +++++++++++++++
9 files changed, 576 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 87729b3..73a3671 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,7 @@ ALL CHANGES:
TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized.
TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts.
TEZ-1578. Remove TeraSort from Tez codebase.
+ TEZ-1569. Add tests for preemption
Release 0.5.1: Unreleased
@@ -33,7 +34,7 @@ ALL CHANGES
TEZ-1534. Make client side configs available to AM and tasks.
TEZ-1574. Support additional formats for the tez deployed archive
-Release 0.5.0: Unreleased
+Release 0.5.0: 2014-09-03
INCOMPATIBLE CHANGES
TEZ-1038. Move TaskLocationHint outside of VertexLocationHint.
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index 13ca2dc..77ab20c 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -123,7 +123,7 @@ public class TezClient {
}
@Private
- TezClient(String name, TezConfiguration tezConf, boolean isSession,
+ protected TezClient(String name, TezConfiguration tezConf, boolean isSession,
@Nullable Map<String, LocalResource> localResources,
@Nullable Credentials credentials) {
this.clientName = name;
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
index 0b615fa..0a95cf0 100644
--- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
+++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.client.api.YarnClientApplication;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -54,6 +55,8 @@ import org.apache.tez.dag.app.DAGAppMasterState;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.utils.EnvironmentUpdateUtils;
+import com.google.common.annotations.VisibleForTesting;
+
public class LocalClient extends FrameworkClient {
public static final Logger LOG = Logger.getLogger(LocalClient.class);
@@ -286,8 +289,7 @@ public class LocalClient extends FrameworkClient {
int nmHttpPort = YarnConfiguration.DEFAULT_NM_WEBAPP_PORT;
long appSubmitTime = System.currentTimeMillis();
- dagAppMaster =
- new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+ dagAppMaster = createDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
new SystemClock(),
appSubmitTime, isSession, userDir.toUri().getPath());
clientHandler = new DAGClientHandler(dagAppMaster);
@@ -305,4 +307,13 @@ public class LocalClient extends FrameworkClient {
return thread;
}
+
+ // this can be overridden by test code to create a mock app
+ @VisibleForTesting
+ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemptId,
+ ContainerId cId, String currentHost, int nmPort, int nmHttpPort,
+ Clock clock, long appSubmitTime, boolean isSession, String userDir) {
+ return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+ new SystemClock(), appSubmitTime, isSession, userDir);
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/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 9cd716a..ea0ab3b 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
@@ -156,12 +156,12 @@ import com.google.common.base.Function;
import com.google.common.collect.Maps;
/**
- * The Map-Reduce Application Master.
+ * The Tez DAG Application Master.
* The state machine is encapsulated in the implementation of Job interface.
* All state changes happens via Job interface. Each event
* results in a Finite State Transition in Job.
*
- * MR AppMaster is the composition of loosely coupled services. The services
+ * Tez DAG AppMaster is the composition of loosely coupled services. The services
* interact with each other via events. The components resembles the
* Actors model. The component acts on received event and send out the
* events to other components.
@@ -443,6 +443,11 @@ public class DAGAppMaster extends AbstractService {
System.exit(0);
}
}
+
+ @VisibleForTesting
+ protected TaskSchedulerEventHandler getTaskSchedulerEventHandler() {
+ return taskSchedulerEventHandler;
+ }
private synchronized void handle(DAGAppMasterEvent event) {
switch (event.getType()) {
@@ -1434,14 +1439,14 @@ public class DAGAppMaster extends AbstractService {
}
return null;
}
-
+
@Override
public synchronized void serviceStart() throws Exception {
//start all the components
startServices();
super.serviceStart();
-
+
// metrics system init is really init & start.
// It's more test friendly to put it here.
DefaultMetricsSystem.initialize("DAGAppMaster");
@@ -1882,7 +1887,7 @@ public class DAGAppMaster extends AbstractService {
UserGroupInformation.setConfiguration(conf);
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
-
+
appMaster.appMasterUgi = UserGroupInformation
.createRemoteUser(jobUserName);
appMaster.appMasterUgi.addCredentials(credentials);
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/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 daaa81b..23f9096 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
@@ -105,6 +105,7 @@ import org.apache.tez.dag.history.events.DAGStartedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption;
import org.apache.tez.dag.utils.RelocalizationUtils;
@@ -690,6 +691,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
return vertex.getVertexStatus(statusOptions);
}
+
+ public TaskAttemptImpl getTaskAttempt(TezTaskAttemptID taId) {
+ return (TaskAttemptImpl) getVertex(taId.getTaskID().getVertexID()).getTask(taId.getTaskID())
+ .getAttempt(taId);
+ }
protected void initializeVerticesAndStart() {
for (Vertex v : vertices.values()) {
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
new file mode 100644
index 0000000..9fe9c4d
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -0,0 +1,245 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.app;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
+import org.apache.tez.dag.app.launcher.ContainerLauncher;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+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.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+import com.google.common.collect.Maps;
+
+@SuppressWarnings("unchecked")
+public class MockDAGAppMaster extends DAGAppMaster {
+
+ MockContainerLauncher containerLauncher;
+
+ // mock container launcher does not launch real tasks.
+ // Upon, launch of a container is simulates the container asking for tasks
+ // Upon receiving a task it simulates completion of the tasks
+ // It can be used to preempt the container for a given task
+ public class MockContainerLauncher extends AbstractService implements ContainerLauncher, Runnable {
+
+ BlockingQueue<NMCommunicatorEvent> eventQueue = new LinkedBlockingQueue<NMCommunicatorEvent>();
+ Thread eventHandlingThread;
+
+ Map<ContainerId, ContainerData> containers = Maps.newConcurrentMap();
+ TaskAttemptListenerImpTezDag taListener;
+
+ AtomicBoolean startScheduling = new AtomicBoolean(true);
+ AtomicBoolean goFlag;
+
+ Map<TezTaskID, Integer> preemptedTasks = Maps.newConcurrentMap();
+
+ public MockContainerLauncher(AtomicBoolean goFlag) {
+ super("MockContainerLauncher");
+ this.goFlag = goFlag;
+ }
+
+ public class ContainerData {
+ ContainerId cId;
+ TezTaskAttemptID taId;
+ String vName;
+ boolean completed;
+
+ public ContainerData(ContainerId cId) {
+ this.cId = cId;
+ }
+
+ void clear() {
+ taId = null;
+ vName = null;
+ completed = false;
+ }
+ }
+
+ @Override
+ public void serviceStart() throws Exception {
+ taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
+ eventHandlingThread = new Thread(this);
+ eventHandlingThread.start();
+ }
+
+ @Override
+ public void serviceStop() throws Exception {
+ if (eventHandlingThread != null) {
+ eventHandlingThread.interrupt();
+ eventHandlingThread.join(2000l);
+ }
+ }
+
+ @Override
+ public void handle(NMCommunicatorEvent event) {
+ switch (event.getType()) {
+ case CONTAINER_LAUNCH_REQUEST:
+ launch((NMCommunicatorLaunchRequestEvent) event);
+ break;
+ case CONTAINER_STOP_REQUEST:
+ stop((NMCommunicatorStopRequestEvent)event);
+ break;
+ }
+ }
+
+
+ void waitToGo() {
+ synchronized (goFlag) {
+ goFlag.set(true);
+ goFlag.notify();
+ try {
+ goFlag.wait();
+ } catch (InterruptedException e) {
+ throw new TezUncheckedException(e);
+ }
+ }
+ }
+
+ public void startScheduling(boolean value) {
+ startScheduling.set(value);
+ }
+
+ public Map<ContainerId, ContainerData> getContainers() {
+ return containers;
+ }
+
+ public void preemptContainerForTask(TezTaskID tId, int uptoVersion) {
+ preemptedTasks.put(tId, uptoVersion);
+ }
+
+ public void preemptContainer(ContainerData cData) {
+ getTaskSchedulerEventHandler().containerCompleted(null,
+ ContainerStatus.newInstance(cData.cId, null, "Preempted", ContainerExitStatus.PREEMPTED));
+ cData.clear();
+ }
+
+ void stop(NMCommunicatorStopRequestEvent event) {
+ // remove from simulated container list
+ containers.remove(event.getContainerId());
+ getContext().getEventHandler().handle(
+ new AMContainerEvent(event.getContainerId(), AMContainerEventType.C_NM_STOP_SENT));
+ }
+
+ void launch(NMCommunicatorLaunchRequestEvent event) {
+ // launch container by putting it in simulated container list
+ containers.put(event.getContainerId(), new ContainerData(event.getContainerId()));
+ getContext().getEventHandler().handle(new AMContainerEventLaunched(event.getContainerId()));
+ }
+
+ @Override
+ public void run() {
+ // wait for test to sync with us and get a reference to us. Go when sync is done
+ waitToGo();
+ while(true) {
+ if (!startScheduling.get()) { // schedule when asked to do so by the test code
+ continue;
+ }
+ for (Map.Entry<ContainerId, ContainerData> entry : containers.entrySet()) {
+ ContainerData cData = entry.getValue();
+ ContainerId cId = entry.getKey();
+ if (cData.taId == null) {
+ // if container is not assigned a task, ask for a task
+ try {
+ ContainerTask cTask = taListener.getTask(new ContainerContext(cId.toString()));
+ if (cTask == null) {
+ continue;
+ }
+ if (cTask.shouldDie()) {
+ containers.remove(cId);
+ } else {
+ cData.taId = cTask.getTaskSpec().getTaskAttemptID();
+ cData.vName = cTask.getTaskSpec().getVertexName();
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ } else if (!cData.completed) {
+ // container is assigned a task and task is not completed
+ // complete the task or preempt the task
+ Integer version = preemptedTasks.get(cData.taId.getTaskID());
+ if (version != null && cData.taId.getId() <= version.intValue()) {
+ preemptContainer(cData);
+ } else {
+ // send a done notification
+ TezVertexID vertexId = cData.taId.getTaskID().getVertexID();
+ cData.completed = true;
+ getContext().getEventHandler().handle(
+ new VertexEventRouteEvent(vertexId, Collections.singletonList(new TezEvent(
+ new TaskAttemptCompletedEvent(), new EventMetaData(
+ EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId)))));
+ cData.clear();
+ }
+ }
+ }
+ try {
+ Thread.sleep(10);
+ } catch (InterruptedException e) {
+ System.out.println("Interrupted in mock container launcher thread");
+ break;
+ }
+ }
+ }
+
+ }
+
+ public MockDAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId,
+ String nmHost, int nmPort, int nmHttpPort, Clock clock, long appSubmitTime,
+ boolean isSession, String workingDirectory, AtomicBoolean launcherGoFlag) {
+ super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
+ isSession, workingDirectory);
+ containerLauncher = new MockContainerLauncher(launcherGoFlag);
+ }
+
+ // use mock container launcher for tests
+ @Override
+ protected ContainerLauncher createContainerLauncher(final AppContext context)
+ throws UnknownHostException {
+ return containerLauncher;
+ }
+
+ public MockContainerLauncher getContainerLauncher() {
+ return containerLauncher;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
new file mode 100644
index 0000000..7e408e1
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
@@ -0,0 +1,48 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.app;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.client.LocalClient;
+
+public class MockLocalClient extends LocalClient {
+ MockDAGAppMaster mockApp;
+ AtomicBoolean mockAppLauncherGoFlag;
+
+ public MockLocalClient(AtomicBoolean mockAppLauncherGoFlag) {
+ this.mockAppLauncherGoFlag = mockAppLauncherGoFlag;
+ }
+
+ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemptId,
+ ContainerId cId, String currentHost, int nmPort, int nmHttpPort,
+ Clock clock, long appSubmitTime, boolean isSession, String userDir) {
+ mockApp = new MockDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+ new SystemClock(), appSubmitTime, isSession, userDir, mockAppLauncherGoFlag);
+ return mockApp;
+ }
+
+ public MockDAGAppMaster getMockApp() {
+ return mockApp;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
new file mode 100644
index 0000000..617415e
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
@@ -0,0 +1,48 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.app;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.client.FrameworkClient;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.TezConfiguration;
+
+public class MockTezClient extends TezClient {
+ MockLocalClient client;
+
+ MockTezClient(String name, TezConfiguration tezConf, boolean isSession,
+ Map<String, LocalResource> localResources, Credentials credentials,
+ AtomicBoolean mockAppLauncherGoFlag) {
+ super(name, tezConf, isSession, localResources, credentials);
+ this.client = new MockLocalClient(mockAppLauncherGoFlag);
+ }
+
+ protected FrameworkClient createFrameworkClient() {
+ return client;
+ }
+
+ public MockLocalClient getLocalClient() {
+ return client;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
new file mode 100644
index 0000000..c7aacd4
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
@@ -0,0 +1,203 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.app;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.EdgeProperty;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.app.MockDAGAppMaster.MockContainerLauncher;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.app.dag.impl.DAGImpl;
+import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.junit.Assert;
+import org.junit.Test;
+
+@SuppressWarnings("deprecation")
+public class TestPreemption {
+
+ static Configuration defaultConf;
+ static FileSystem localFs;
+ static Path workDir;
+
+ static {
+ try {
+ defaultConf = new Configuration(false);
+ defaultConf.set("fs.defaultFS", "file:///");
+ defaultConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true);
+ localFs = FileSystem.getLocal(defaultConf);
+ workDir = new Path(new Path(System.getProperty("test.build.data", "/tmp")),
+ "TestDAGAppMaster").makeQualified(localFs);
+ } catch (IOException e) {
+ throw new RuntimeException("init failure", e);
+ }
+ }
+
+ MockDAGAppMaster mockApp;
+ MockContainerLauncher mockLauncher;
+
+ int dagCount = 0;
+
+ DAG createDAG(DataMovementType dmType) {
+ DAG dag = DAG.create("test-" + dagCount++);
+ Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5);
+ Vertex vB = Vertex.create("B", ProcessorDescriptor.create("Proc.class"), 5);
+ Edge eAB = Edge.create(vA, vB,
+ EdgeProperty.create(dmType, DataSourceType.PERSISTED,
+ SchedulingType.SEQUENTIAL, OutputDescriptor.create("O.class"),
+ InputDescriptor.create("I.class")));
+
+ dag.addVertex(vA).addVertex(vB).addEdge(eAB);
+ return dag;
+ }
+
+ @Test
+ public void testPreemptionWithoutSession() throws Exception {
+ System.out.println("TestPreemptionWithoutSession");
+ TezConfiguration tezconf = new TezConfiguration(defaultConf);
+ tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+ AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false);
+ MockTezClient tezClient = new MockTezClient("testPreemption", tezconf, false, null, null,
+ mockAppLauncherGoFlag);
+ tezClient.start();
+
+ DAGClient dagClient = tezClient.submitDAG(createDAG(DataMovementType.SCATTER_GATHER));
+ // now the MockApp has been started. sync with it to get the launcher
+ syncWithMockAppLauncher(false, mockAppLauncherGoFlag, tezClient);
+
+ DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
+ int vertexIndex = 0;
+ int upToTaskVersion = 3;
+ TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), vertexIndex);
+ TezTaskAttemptID taId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0);
+
+ mockLauncher.preemptContainerForTask(taId.getTaskID(), upToTaskVersion);
+ mockLauncher.startScheduling(true);
+
+ dagClient.waitForCompletion();
+ Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+
+ for (int i=0; i<=upToTaskVersion; ++i) {
+ TezTaskAttemptID testTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), i);
+ TaskAttemptImpl taImpl = dagImpl.getTaskAttempt(testTaId);
+ Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState());
+ }
+
+ tezClient.stop();
+ }
+
+ @Test
+ public void testPreemptionWithSession() throws Exception {
+ System.out.println("TestPreemptionWithSession");
+ MockTezClient tezClient = createTezSession();
+ testPreemptionSingle(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 0, "Scatter-Gather");
+ testPreemptionMultiple(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 0, "Scatter-Gather");
+ testPreemptionSingle(tezClient, createDAG(DataMovementType.BROADCAST), 0, "Broadcast");
+ testPreemptionMultiple(tezClient, createDAG(DataMovementType.BROADCAST), 0, "Broadcast");
+ testPreemptionSingle(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 0, "1-1");
+ testPreemptionMultiple(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 0, "1-1");
+ testPreemptionSingle(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 1, "Scatter-Gather");
+ testPreemptionMultiple(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 1, "Scatter-Gather");
+ testPreemptionSingle(tezClient, createDAG(DataMovementType.BROADCAST), 1, "Broadcast");
+ testPreemptionMultiple(tezClient, createDAG(DataMovementType.BROADCAST), 1, "Broadcast");
+ testPreemptionSingle(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 1, "1-1");
+ testPreemptionMultiple(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 1, "1-1");
+ tezClient.stop();
+ }
+
+ MockTezClient createTezSession() throws Exception {
+ TezConfiguration tezconf = new TezConfiguration(defaultConf);
+ tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+ AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false);
+ MockTezClient tezClient = new MockTezClient("testPreemption", tezconf, true, null, null,
+ mockAppLauncherGoFlag);
+ tezClient.start();
+ syncWithMockAppLauncher(false, mockAppLauncherGoFlag, tezClient);
+ return tezClient;
+ }
+
+ void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLauncherGoFlag,
+ MockTezClient tezClient) throws Exception {
+ synchronized (mockAppLauncherGoFlag) {
+ while (!mockAppLauncherGoFlag.get()) {
+ mockAppLauncherGoFlag.wait();
+ }
+ mockApp = tezClient.getLocalClient().getMockApp();
+ mockLauncher = mockApp.getContainerLauncher();
+ mockLauncher.startScheduling(allowScheduling);
+ mockAppLauncherGoFlag.notify();
+ }
+ }
+
+ void testPreemptionSingle(MockTezClient tezClient, DAG dag, int vertexIndex, String info)
+ throws Exception {
+ testPreemptionJob(tezClient, dag, vertexIndex, 0, info + "-Single");
+ }
+
+ void testPreemptionMultiple(MockTezClient tezClient, DAG dag, int vertexIndex, String info)
+ throws Exception {
+ testPreemptionJob(tezClient, dag, vertexIndex, 3, info + "-Multiple");
+ }
+
+ void testPreemptionJob(MockTezClient tezClient, DAG dag, int vertexIndex,
+ int upToTaskVersion, String info) throws Exception {
+ System.out.println("TestPreemption - Running - " + info);
+ TezConfiguration tezconf = new TezConfiguration(defaultConf);
+ tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+
+ mockLauncher.startScheduling(false); // turn off scheduling to block DAG before submitting it
+ DAGClient dagClient = tezClient.submitDAG(dag);
+
+ DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
+ TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), vertexIndex);
+ TezTaskAttemptID taId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0);
+
+ mockLauncher.preemptContainerForTask(taId.getTaskID(), upToTaskVersion);
+ mockLauncher.startScheduling(true);
+
+ dagClient.waitForCompletion();
+ Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+
+ for (int i=0; i<=upToTaskVersion; ++i) {
+ TezTaskAttemptID testTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), i);
+ TaskAttemptImpl taImpl = dagImpl.getTaskAttempt(testTaId);
+ Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState());
+ }
+
+ System.out.println("TestPreemption - Done running - " + info);
+ }
+}
[07/25] git commit: TEZ-1345. Add checks to guarantee all init events
are written to recovery to consider vertex initialized. (Jeff Zhang via
hitesh)
Posted by je...@apache.org.
TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized. (Jeff Zhang via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/fb05aace
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/fb05aace
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/fb05aace
Branch: refs/heads/TEZ-8
Commit: fb05aace5062dc441da80f5ac8d4e322ab2a15ef
Parents: 16a0f57
Author: Hitesh Shah <hi...@apache.org>
Authored: Thu Sep 11 22:29:29 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Thu Sep 11 22:29:29 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 7 +-
.../tez/dag/app/dag/impl/VertexManager.java | 28 ++++----
.../tez/dag/app/dag/impl/TestVertexManager.java | 76 ++++++++++++++++++++
.../org/apache/tez/test/TestDAGRecovery.java | 67 ++++++++++++++++-
.../apache/tez/test/dag/MultiAttemptDAG.java | 35 ++++++++-
6 files changed, 195 insertions(+), 19 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 519aaa6..f29d48d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -11,6 +11,7 @@ ALL CHANGES:
TEZ-1559. Add system tests for AM recovery.
TEZ-850. Recovery unit tests.
TEZ-853. Support counters recovery.
+ TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/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 31240cb..6437e5b 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
@@ -156,7 +156,6 @@ import org.apache.tez.runtime.api.impl.GroupInputSpec;
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 com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.HashMultiset;
@@ -2782,10 +2781,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
VertexEventRootInputInitialized liInitEvent = (VertexEventRootInputInitialized) event;
VertexState state = vertex.getState();
if (state == VertexState.INITIALIZING) {
- vertex.vertexManager.onRootVertexInitialized(
+ List<TezEvent> inputInfoEvents =
+ vertex.vertexManager.onRootVertexInitialized(
liInitEvent.getInputName(),
vertex.getAdditionalInputs().get(liInitEvent.getInputName())
.getIODescriptor(), liInitEvent.getEvents());
+ if (inputInfoEvents != null && !inputInfoEvents.isEmpty()) {
+ ROUTE_EVENT_TRANSITION.transition(vertex, new VertexEventRouteEvent(vertex.vertexId, inputInfoEvents));
+ }
}
vertex.numInitializedInputs++;
http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index 177b946..c2ff660 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -24,6 +24,7 @@ import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
import javax.annotation.Nullable;
@@ -44,7 +45,6 @@ import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.dag.TaskAttempt;
import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezTaskID;
import org.apache.tez.runtime.api.Event;
@@ -68,10 +68,11 @@ public class VertexManager {
VertexManagerPluginContextImpl pluginContext;
UserPayload payload = null;
AppContext appContext;
-
+ ConcurrentHashMap<String, List<TezEvent>> cachedRootInputEventMap;
+
class VertexManagerPluginContextImpl implements VertexManagerPluginContext {
// TODO Add functionality to allow VertexManagers to send VertexManagerEvents
-
+
private EventMetaData rootEventSourceMetadata = new EventMetaData(EventProducerConsumerType.INPUT,
managedVertex.getName(), "NULL_VERTEX", null);
private Map<String, EventMetaData> destinationEventMetadataMap = Maps.newHashMap();
@@ -80,7 +81,7 @@ public class VertexManager {
public Map<String, EdgeProperty> getInputVertexEdgeProperties() {
// TODO Something similar for Initial Inputs - payload etc visible
Map<Vertex, Edge> inputs = managedVertex.getInputVertices();
- Map<String, EdgeProperty> vertexEdgeMap =
+ Map<String, EdgeProperty> vertexEdgeMap =
Maps.newHashMapWithExpectedSize(inputs.size());
for (Map.Entry<Vertex, Edge> entry : inputs.entrySet()) {
vertexEdgeMap.put(entry.getKey().getName(), entry.getValue().getEdgeProperty());
@@ -115,7 +116,7 @@ public class VertexManager {
@Override
public Set<String> getVertexInputNames() {
Set<String> inputNames = null;
- Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
+ Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
inputs = managedVertex.getAdditionalInputs();
if (inputs != null) {
inputNames = inputs.keySet();
@@ -128,7 +129,6 @@ public class VertexManager {
return payload;
}
- @SuppressWarnings("unchecked")
@Override
public void addRootInputEvents(final String inputName,
Collection<InputDataInformationEvent> events) {
@@ -142,8 +142,8 @@ public class VertexManager {
return tezEvent;
}
});
- appContext.getEventHandler().handle(
- new VertexEventRouteEvent(managedVertex.getVertexId(), Lists.newArrayList(tezEvents)));
+
+ cachedRootInputEventMap.put(inputName,Lists.newArrayList(tezEvents));
// Recovery handling is taken care of by the Vertex.
}
@@ -201,7 +201,7 @@ public class VertexManager {
}
}
- public VertexManager(VertexManagerPluginDescriptor pluginDesc,
+ public VertexManager(VertexManagerPluginDescriptor pluginDesc,
Vertex managedVertex, AppContext appContext) {
checkNotNull(pluginDesc, "pluginDesc is null");
checkNotNull(managedVertex, "managedVertex is null");
@@ -209,12 +209,13 @@ public class VertexManager {
this.pluginDesc = pluginDesc;
this.managedVertex = managedVertex;
this.appContext = appContext;
+ this.cachedRootInputEventMap = new ConcurrentHashMap<String, List<TezEvent>>();
}
-
+
public VertexManagerPlugin getPlugin() {
return plugin;
}
-
+
public void initialize() {
pluginContext = new VertexManagerPluginContextImpl();
if (pluginDesc != null) {
@@ -246,7 +247,7 @@ public class VertexManager {
public void onSourceTaskCompleted(TezTaskID tezTaskId) {
Integer taskId = new Integer(tezTaskId.getId());
- String vertexName =
+ String vertexName =
appContext.getCurrentDAG().getVertex(tezTaskId.getVertexID()).getName();
plugin.onSourceTaskCompleted(vertexName, taskId);
}
@@ -255,8 +256,9 @@ public class VertexManager {
plugin.onVertexManagerEventReceived(vmEvent);
}
- public void onRootVertexInitialized(String inputName,
+ public List<TezEvent> onRootVertexInitialized(String inputName,
InputDescriptor inputDescriptor, List<Event> events) {
plugin.onRootVertexInitialized(inputName, inputDescriptor, events);
+ return cachedRootInputEventMap.get(inputName);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
new file mode 100644
index 0000000..b3e66bc
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.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.dag.app.dag.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.VertexManagerPluginDescriptor;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Test;
+
+public class TestVertexManager {
+
+ @Test
+ public void testOnRootVertexInitialized() {
+ Vertex mockVertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
+ AppContext mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+ doReturn("vertex1").when(mockVertex).getName();
+ when(
+ mockAppContext.getCurrentDAG().getVertex(any(String.class))
+ .getTotalTasks()).thenReturn(1);
+
+ VertexManager vm =
+ new VertexManager(
+ VertexManagerPluginDescriptor.create(RootInputVertexManager.class
+ .getName()), mockVertex, mockAppContext);
+ vm.initialize();
+ InputDescriptor id1 = mock(InputDescriptor.class);
+ List<Event> events1 = new LinkedList<Event>();
+ InputDataInformationEvent diEvent1 =
+ InputDataInformationEvent.createWithSerializedPayload(0, null);
+ events1.add(diEvent1);
+ List<TezEvent> tezEvents1 =
+ vm.onRootVertexInitialized("input1", id1, events1);
+ assertEquals(1, tezEvents1.size());
+ assertEquals(diEvent1, tezEvents1.get(0).getEvent());
+
+ InputDescriptor id2 = mock(InputDescriptor.class);
+ List<Event> events2 = new LinkedList<Event>();
+ InputDataInformationEvent diEvent2 =
+ InputDataInformationEvent.createWithSerializedPayload(0, null);
+ events2.add(diEvent2);
+ List<TezEvent> tezEvents2 =
+ vm.onRootVertexInitialized("input1", id2, events2);
+ assertEquals(tezEvents2.size(), 1);
+ assertEquals(diEvent2, tezEvents2.get(0).getEvent());
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
index c9acdc2..7676313 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
@@ -24,21 +24,30 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.tez.client.TezClientUtils;
import org.apache.tez.client.TezClient;
+import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.dag.api.DAG;
import org.apache.tez.dag.api.DataSourceDescriptor;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.InputInitializerDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.client.DAGClient;
import org.apache.tez.dag.api.client.DAGStatus;
import org.apache.tez.dag.api.client.DAGStatus.State;
+import org.apache.tez.dag.app.RecoveryParser;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
import org.apache.tez.test.dag.MultiAttemptDAG;
import org.apache.tez.test.dag.MultiAttemptDAG.FailingInputInitializer;
import org.apache.tez.test.dag.MultiAttemptDAG.NoOpInput;
+import org.apache.tez.test.dag.MultiAttemptDAG.TestRootInputInitializer;
import org.apache.tez.test.dag.SimpleVTestDAG;
import org.junit.After;
import org.junit.AfterClass;
@@ -48,6 +57,7 @@ import org.junit.BeforeClass;
import org.junit.Test;
import java.io.IOException;
+import java.util.List;
import java.util.Random;
public class TestDAGRecovery {
@@ -61,6 +71,7 @@ public class TestDAGRecovery {
private static MiniDFSCluster dfsCluster = null;
private static TezClient tezSession = null;
private static FileSystem remoteFs = null;
+ private static TezConfiguration tezConf = null;
@BeforeClass
public static void beforeClass() throws Exception {
@@ -120,7 +131,7 @@ public class TestDAGRecovery {
.valueOf(new Random().nextInt(100000))));
TezClientUtils.ensureStagingDirExists(conf, remoteStagingDir);
- TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+ tezConf = new TezConfiguration(miniTezCluster.getConfig());
tezConf.setInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS, 0);
tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "DEBUG");
tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
@@ -130,6 +141,7 @@ public class TestDAGRecovery {
tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
+ tezConf.set(TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, "false");
tezSession = TezClient.create("TestDAGRecovery", tezConf);
tezSession.start();
@@ -165,13 +177,64 @@ public class TestDAGRecovery {
Assert.assertEquals(finalState, dagStatus.getState());
}
+ private void verifyRecoveryLog() throws IOException{
+ ApplicationId appId = tezSession.getAppMasterApplicationId();
+ Path tezSystemStagingDir = TezCommonUtils.getTezSystemStagingPath(tezConf, appId.toString());
+ Path recoveryDataDir = TezCommonUtils.getRecoveryPath(tezSystemStagingDir, tezConf);
+
+ FileSystem fs = tezSystemStagingDir.getFileSystem(tezConf);
+ for (int i=1; i<=3; ++i) {
+ Path currentAttemptRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir,i);
+ Path recoveryFilePath = new Path(currentAttemptRecoveryDataDir,
+ appId.toString().replace("application", "dag") + "_1" + TezConstants.DAG_RECOVERY_RECOVER_FILE_SUFFIX);
+ List<HistoryEvent> historyEvents = RecoveryParser.parseDAGRecoveryFile(
+ fs.open(recoveryFilePath));
+
+ int inputInfoEventIndex = -1;
+ int vertexInitedEventIndex = -1;
+ for (int j=0;j<historyEvents.size(); ++j) {
+ HistoryEvent historyEvent = historyEvents.get(j);
+ LOG.info("Parsed event from recovery stream"
+ + ", eventType=" + historyEvent.getEventType()
+ + ", event=" + historyEvent);
+ if (historyEvent.getEventType() == HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED) {
+ VertexDataMovementEventsGeneratedEvent dmEvent =
+ (VertexDataMovementEventsGeneratedEvent)historyEvent;
+ // TODO do not need to check whether it is -1 after Tez-1521 is resolved
+ if (dmEvent.getVertexID().getId() == 0 && inputInfoEventIndex == -1) {
+ inputInfoEventIndex = j;
+ }
+ }
+ if (historyEvent.getEventType() == HistoryEventType.VERTEX_INITIALIZED) {
+ VertexInitializedEvent vInitedEvent = (VertexInitializedEvent) historyEvent;
+ if (vInitedEvent.getVertexID().getId() == 0) {
+ vertexInitedEventIndex = j;
+ }
+ }
+ }
+ // v1's init events must be logged before its VertexInitializedEvent (Tez-1345)
+ Assert.assertTrue("can not find VERTEX_DATA_MOVEMENT_EVENTS_GENERATED for v1", inputInfoEventIndex != -1);
+ Assert.assertTrue("can not find VERTEX_INITIALIZED for v1", vertexInitedEventIndex != -1);
+ Assert.assertTrue("VERTEX_DATA_MOVEMENT_EVENTS_GENERATED is logged before VERTEX_INITIALIZED for v1",
+ inputInfoEventIndex < vertexInitedEventIndex);
+ }
+ }
+
@Test(timeout=120000)
public void testBasicRecovery() throws Exception {
DAG dag = MultiAttemptDAG.createDAG("TestBasicRecovery", null);
+ // add input to v1 to make sure that there will be init events for v1 (TEZ-1345)
+ DataSourceDescriptor dataSource =
+ DataSourceDescriptor.create(InputDescriptor.create(NoOpInput.class.getName()),
+ InputInitializerDescriptor.create(TestRootInputInitializer.class.getName()), null);
+ dag.getVertex("v1").addDataSource("Input", dataSource);
+
runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ verifyRecoveryLog();
+
// it should fail if submitting same dags in recovery mode (TEZ-1064)
- try{
+ try {
DAGClient dagClient = tezSession.submitDAG(dag);
Assert.fail("Expected DAG submit to fail on duplicate dag name");
} catch (TezException e) {
http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
index 7fc9ad7..58b9413 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
@@ -22,6 +22,7 @@ import com.google.common.collect.Lists;
import com.google.common.primitives.Ints;
import java.nio.ByteBuffer;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -53,12 +54,15 @@ import org.apache.tez.runtime.api.OutputContext;
import org.apache.tez.runtime.api.InputInitializer;
import org.apache.tez.runtime.api.InputInitializerContext;
import org.apache.tez.runtime.api.Writer;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
import org.apache.tez.runtime.api.events.InputInitializerEvent;
import org.apache.tez.runtime.api.events.VertexManagerEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
import org.apache.tez.test.TestInput;
import org.apache.tez.test.TestOutput;
import org.apache.tez.test.TestProcessor;
+import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@@ -148,8 +152,15 @@ public class MultiAttemptDAG {
}
@Override
- public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor, List<Event> events) {
- // Do nothing
+ public void onRootVertexInitialized(String inputName,
+ InputDescriptor inputDescriptor, List<Event> events) {
+ List<InputDataInformationEvent> inputInfoEvents = new ArrayList<InputDataInformationEvent>();
+ for (Event event: events) {
+ if (event instanceof InputDataInformationEvent) {
+ inputInfoEvents.add((InputDataInformationEvent)event);
+ }
+ }
+ getContext().addRootInputEvents(inputName, inputInfoEvents);
}
}
@@ -214,6 +225,26 @@ public class MultiAttemptDAG {
}
}
+ public static class TestRootInputInitializer extends InputInitializer {
+
+ public TestRootInputInitializer(InputInitializerContext initializerContext) {
+ super(initializerContext);
+ }
+
+ @Override
+ public List<Event> initialize() throws Exception {
+ List<Event> events = new ArrayList<Event>();
+ events.add(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.allocate(0)));
+ return events;
+ }
+
+ @Override
+ public void handleInputInitializerEvent(List<InputInitializerEvent> events)
+ throws Exception {
+ throw new UnsupportedOperationException("Not supported");
+ }
+ }
+
public static class FailingInputInitializer extends InputInitializer {
public FailingInputInitializer(InputInitializerContext initializerContext) {
[17/25] git commit: TEZ-1524. Resolve user group information only if
ACLs are enabled (gopalv)
Posted by je...@apache.org.
TEZ-1524. Resolve user group information only if ACLs are enabled (gopalv)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/edb841c0
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/edb841c0
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/edb841c0
Branch: refs/heads/TEZ-8
Commit: edb841c08de123ff2c5ace0662ae78bf3c58f2c0
Parents: 9dd0cb4
Author: Gopal V <go...@apache.org>
Authored: Fri Sep 12 15:04:32 2014 -0700
Committer: Gopal V <go...@apache.org>
Committed: Fri Sep 12 15:04:32 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/tez/common/security/ACLManager.java | 30 +-
.../tez/common/security/TestACLManager.java | 417 ++++++++++---------
...DAGClientAMProtocolBlockingPBServerImpl.java | 40 +-
4 files changed, 250 insertions(+), 238 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f71c2e2..59be260 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,7 @@ ALL CHANGES:
TEZ-1578. Remove TeraSort from Tez codebase.
TEZ-1569. Add tests for preemption
TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
+ TEZ-1524. Resolve user group information only if ACLs are enabled.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
index d8be327..77ab065 100644
--- a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
+++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
@@ -18,6 +18,7 @@
package org.apache.tez.common.security;
+import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
@@ -29,6 +30,7 @@ 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.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.tez.dag.api.TezConfiguration;
@@ -92,13 +94,19 @@ public class ACLManager {
}
@VisibleForTesting
- boolean checkAccess(String user, Collection<String> userGroups, ACLType aclType) {
+ boolean checkAccess(UserGroupInformation ugi, ACLType aclType) {
+
if (!aclsEnabled) {
return true;
}
+
+ String user = ugi.getShortUserName();
+ Collection<String> userGroups = Arrays.asList(ugi.getGroupNames());
+
if (amUser.equals(user)) {
return true;
}
+
if (EnumSet.of(ACLType.DAG_MODIFY_ACL, ACLType.DAG_VIEW_ACL).contains(aclType)) {
if (dagUser != null && dagUser.equals(user)) {
return true;
@@ -129,22 +137,22 @@ public class ACLManager {
return false;
}
- public boolean checkAMViewAccess(String user, Collection<String> userGroups) {
- return checkAccess(user, userGroups, ACLType.AM_VIEW_ACL);
+ public boolean checkAMViewAccess(UserGroupInformation ugi) {
+ return checkAccess(ugi, ACLType.AM_VIEW_ACL);
}
- public boolean checkAMModifyAccess(String user, Collection<String> userGroups) {
- return checkAccess(user, userGroups, ACLType.AM_MODIFY_ACL);
+ public boolean checkAMModifyAccess(UserGroupInformation ugi) {
+ return checkAccess(ugi, ACLType.AM_MODIFY_ACL);
}
- public boolean checkDAGViewAccess(String user, Collection<String> userGroups) {
- return checkAccess(user, userGroups, ACLType.AM_VIEW_ACL)
- || checkAccess(user, userGroups, ACLType.DAG_VIEW_ACL);
+ public boolean checkDAGViewAccess(UserGroupInformation ugi) {
+ return checkAccess(ugi, ACLType.AM_VIEW_ACL)
+ || checkAccess(ugi, ACLType.DAG_VIEW_ACL);
}
- public boolean checkDAGModifyAccess(String user, Collection<String> userGroups) {
- return checkAccess(user, userGroups, ACLType.AM_MODIFY_ACL)
- || checkAccess(user, userGroups, ACLType.DAG_MODIFY_ACL);
+ public boolean checkDAGModifyAccess(UserGroupInformation ugi) {
+ return checkAccess(ugi, ACLType.AM_MODIFY_ACL)
+ || checkAccess(ugi, ACLType.DAG_MODIFY_ACL);
}
public Map<ApplicationAccessType, String> toYARNACls() {
http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
index 7ad4ede..bc35b51 100644
--- a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
+++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
@@ -23,6 +23,7 @@ import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezConstants;
@@ -33,138 +34,141 @@ import com.google.common.collect.Sets;
public class TestACLManager {
- private static final Set<String> noGroups = Sets.newHashSet();
+ private static final String[] noGroups = new String[0];
@Test
public void testCurrentUserACLChecks() {
- String currentUser = "currentUser";
- ACLManager aclManager = new ACLManager(currentUser);
+ UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+ UserGroupInformation dagUser = UserGroupInformation.createUserForTesting("dagUser", noGroups);
+ UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", noGroups);
- String user = "user1";
- Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+ ACLManager aclManager = new ACLManager(currentUser.getShortUserName());
+
+ UserGroupInformation user = user1;
+
+ Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
user = currentUser;
- Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
- Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
- aclManager = new ACLManager(currentUser, new Configuration(false));
+ aclManager = new ACLManager(currentUser.getShortUserName(), new Configuration(false));
- user = "user1";
- Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+ user = user1;
+ Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
user = currentUser;
- Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
- Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
- String dagUser = "dagUser";
- ACLManager dagAclManager = new ACLManager(aclManager, dagUser, new Configuration(false));
+ ACLManager dagAclManager = new ACLManager(aclManager, dagUser.getShortUserName(), new Configuration(false));
user = dagUser;
- Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
- Assert.assertTrue(dagAclManager.checkAccess(user, null, ACLType.DAG_VIEW_ACL));
- Assert.assertTrue(dagAclManager.checkAccess(user, null, ACLType.DAG_MODIFY_ACL));
- user = "user1";
- Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.DAG_VIEW_ACL));
- Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.DAG_MODIFY_ACL));
+ Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
+ Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL));
+ Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL));
+ user = user1;
+ Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL));
+ Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL));
}
@Test
public void testOtherUserACLChecks() throws IOException {
- Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
- Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
- Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
- String currentUser = "currentUser";
- String user1 = "user1"; // belongs to grp1 and grp2
- String user2 = "user2"; // belongs to grp3 and grp4
- String user3 = "user3";
- String user4 = "user4";
- String user5 = "user5"; // belongs to grp5 and grp6
- String user6 = "user6";
+ String[] groups1 = new String[] {"grp1", "grp2"};
+ String[] groups2 = new String[] {"grp3", "grp4"};
+ String[] groups3 = new String[] {"grp5", "grp6"};
+
+ UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+ UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+ UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+ UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+ UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+ UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+ UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
Configuration conf = new Configuration(false);
// View ACLs: user1, user4, grp3, grp4.
- String viewACLs = user1 + "," + user4
+ String viewACLs = user1.getShortUserName() + "," + user4.getShortUserName()
+ " " + "grp3,grp4 ";
// Modify ACLs: user3, grp6, grp7
- String modifyACLs = user3 + " " + "grp6,grp7";
+ String modifyACLs = user3.getShortUserName() + " " + "grp6,grp7";
conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
- ACLManager aclManager = new ACLManager(currentUser, conf);
-
- Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_VIEW_ACL));
- Assert.assertTrue(aclManager.checkAccess(user1, groups1, ACLType.AM_VIEW_ACL));
- Assert.assertTrue(aclManager.checkAccess(user2, groups2, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user3, noGroups, ACLType.AM_VIEW_ACL));
- Assert.assertTrue(aclManager.checkAccess(user4, noGroups, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_VIEW_ACL));
-
- Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user1, groups1, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_MODIFY_ACL));
- Assert.assertTrue(aclManager.checkAccess(user3, noGroups, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user4, noGroups, ACLType.AM_MODIFY_ACL));
- Assert.assertTrue(aclManager.checkAccess(user5, groups3, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_MODIFY_ACL));
+ ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
+
+ Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL));
+
+ Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL));
}
@Test
public void testNoGroupsACLChecks() throws IOException {
- Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
- Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
- Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
- String currentUser = "currentUser";
- String user1 = "user1"; // belongs to grp1 and grp2
- String user2 = "user2"; // belongs to grp3 and grp4
- String user3 = "user3";
- String user4 = "user4";
- String user5 = "user5"; // belongs to grp5 and grp6
- String user6 = "user6";
+ String[] groups1 = new String[] {"grp1", "grp2"};
+ String[] groups2 = new String[] {"grp3", "grp4"};
+ String[] groups3 = new String[] {"grp5", "grp6"};
+
+ UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+ UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+ UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+ UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+ UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+ UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+ UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
Configuration conf = new Configuration(false);
// View ACLs: user1, user4
- String viewACLs = user1 + "," + user4 + " ";
+ String viewACLs = user1.getShortUserName() + "," + user4.getShortUserName() + " ";
// Modify ACLs: user3
- String modifyACLs = "user3 ";
+ String modifyACLs = user3.getShortUserName() + " ";
conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
- ACLManager aclManager = new ACLManager(currentUser, conf);
- Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_VIEW_ACL));
- Assert.assertTrue(aclManager.checkAccess(user1, groups1, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user3, noGroups, ACLType.AM_VIEW_ACL));
- Assert.assertTrue(aclManager.checkAccess(user4, noGroups, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_VIEW_ACL));
- Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_VIEW_ACL));
-
- Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user1, groups1, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_MODIFY_ACL));
- Assert.assertTrue(aclManager.checkAccess(user3, noGroups, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user4, noGroups, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_MODIFY_ACL));
- Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_MODIFY_ACL));
+ ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
+ Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_VIEW_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL));
+
+ Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL));
+ Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL));
+ Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL));
}
@Test
public void checkAMACLs() throws IOException {
- Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
- Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
- Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
- String currentUser = "currentUser";
- String user1 = "user1"; // belongs to grp1 and grp2
- String user2 = "user2"; // belongs to grp3 and grp4
- String user3 = "user3";
- String user4 = "user4";
- String user5 = "user5"; // belongs to grp5 and grp6
- String user6 = "user6";
+ String[] groups1 = new String[] {"grp1", "grp2"};
+ String[] groups2 = new String[] {"grp3", "grp4"};
+ String[] groups3 = new String[] {"grp5", "grp6"};
+
+ UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+ UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+ UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+ UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+ UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+ UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+ UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
Configuration conf = new Configuration(false);
// View ACLs: user1, user4, grp3, grp4.
@@ -174,55 +178,55 @@ public class TestACLManager {
conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
- ACLManager aclManager = new ACLManager(currentUser, conf);
-
- Assert.assertTrue(aclManager.checkAMViewAccess(currentUser, null));
- Assert.assertTrue(aclManager.checkAMViewAccess(user1, groups1));
- Assert.assertTrue(aclManager.checkAMViewAccess(user2, groups2));
- Assert.assertFalse(aclManager.checkAMViewAccess(user3, noGroups));
- Assert.assertTrue(aclManager.checkAMViewAccess(user4, noGroups));
- Assert.assertFalse(aclManager.checkAMViewAccess(user5, groups3));
- Assert.assertFalse(aclManager.checkAMViewAccess(user6, noGroups));
-
- Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser, null));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user1, groups1));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user2, groups2));
- Assert.assertTrue(aclManager.checkAMModifyAccess(user3, noGroups));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user4, noGroups));
- Assert.assertTrue(aclManager.checkAMModifyAccess(user5, groups3));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user6, noGroups));
-
- Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser, null));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user1, groups1));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user2, groups2));
- Assert.assertFalse(aclManager.checkDAGViewAccess(user3, noGroups));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user4, noGroups));
- Assert.assertFalse(aclManager.checkDAGViewAccess(user5, groups3));
- Assert.assertFalse(aclManager.checkDAGViewAccess(user6, noGroups));
-
- Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser, null));
- Assert.assertFalse(aclManager.checkDAGModifyAccess(user1, groups1));
- Assert.assertFalse(aclManager.checkDAGModifyAccess(user2, groups2));
- Assert.assertTrue(aclManager.checkDAGModifyAccess(user3, noGroups));
- Assert.assertFalse(aclManager.checkDAGModifyAccess(user4, noGroups));
- Assert.assertTrue(aclManager.checkDAGModifyAccess(user5, groups3));
- Assert.assertFalse(aclManager.checkDAGModifyAccess(user6, noGroups));
+ ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
+
+ Assert.assertTrue(aclManager.checkAMViewAccess(currentUser));
+ Assert.assertTrue(aclManager.checkAMViewAccess(user1));
+ Assert.assertTrue(aclManager.checkAMViewAccess(user2));
+ Assert.assertFalse(aclManager.checkAMViewAccess(user3));
+ Assert.assertTrue(aclManager.checkAMViewAccess(user4));
+ Assert.assertFalse(aclManager.checkAMViewAccess(user5));
+ Assert.assertFalse(aclManager.checkAMViewAccess(user6));
+
+ Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user1));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user2));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(user3));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user4));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(user5));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user6));
+
+ Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user1));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user2));
+ Assert.assertFalse(aclManager.checkDAGViewAccess(user3));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user4));
+ Assert.assertFalse(aclManager.checkDAGViewAccess(user5));
+ Assert.assertFalse(aclManager.checkDAGViewAccess(user6));
+
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser));
+ Assert.assertFalse(aclManager.checkDAGModifyAccess(user1));
+ Assert.assertFalse(aclManager.checkDAGModifyAccess(user2));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(user3));
+ Assert.assertFalse(aclManager.checkDAGModifyAccess(user4));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(user5));
+ Assert.assertFalse(aclManager.checkDAGModifyAccess(user6));
}
@Test
public void checkDAGACLs() throws IOException {
- Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
- Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
- Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
- String currentUser = "currentUser";
- String user1 = "user1"; // belongs to grp1 and grp2
- String user2 = "user2"; // belongs to grp3 and grp4
- String user3 = "user3";
- String user4 = "user4";
- String user5 = "user5"; // belongs to grp5 and grp6
- String user6 = "user6";
+ String[] groups1 = new String[] {"grp1", "grp2"};
+ String[] groups2 = new String[] {"grp3", "grp4"};
+ String[] groups3 = new String[] {"grp5", "grp6"};
+
+ UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+ UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+ UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+ UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+ UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+ UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+ UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
Configuration conf = new Configuration(false);
// View ACLs: user1, user4, grp3, grp4.
@@ -239,46 +243,46 @@ public class TestACLManager {
conf.set(TezConstants.TEZ_DAG_VIEW_ACLS, dagViewACLs);
conf.set(TezConstants.TEZ_DAG_MODIFY_ACLS, dagModifyACLs);
- String dagUser = "dagUser";
-
- ACLManager amAclManager = new ACLManager(currentUser, conf);
- ACLManager aclManager = new ACLManager(amAclManager, dagUser, conf);
-
- Assert.assertTrue(aclManager.checkAMViewAccess(currentUser, null));
- Assert.assertFalse(aclManager.checkAMViewAccess(dagUser, null));
- Assert.assertTrue(aclManager.checkAMViewAccess(user1, groups1));
- Assert.assertTrue(aclManager.checkAMViewAccess(user2, groups2));
- Assert.assertFalse(aclManager.checkAMViewAccess(user3, noGroups));
- Assert.assertTrue(aclManager.checkAMViewAccess(user4, noGroups));
- Assert.assertFalse(aclManager.checkAMViewAccess(user5, groups3));
- Assert.assertFalse(aclManager.checkAMViewAccess(user6, noGroups));
-
- Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser, null));
- Assert.assertFalse(aclManager.checkAMModifyAccess(dagUser, null));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user1, groups1));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user2, groups2));
- Assert.assertTrue(aclManager.checkAMModifyAccess(user3, noGroups));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user4, noGroups));
- Assert.assertTrue(aclManager.checkAMModifyAccess(user5, groups3));
- Assert.assertFalse(aclManager.checkAMModifyAccess(user6, noGroups));
-
- Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser, null));
- Assert.assertTrue(aclManager.checkDAGViewAccess(dagUser, null));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user1, groups1));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user2, groups2));
- Assert.assertFalse(aclManager.checkDAGViewAccess(user3, noGroups));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user4, noGroups));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user5, groups3));
- Assert.assertTrue(aclManager.checkDAGViewAccess(user6, noGroups));
-
- Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser, null));
- Assert.assertTrue(aclManager.checkDAGModifyAccess(dagUser, null));
- Assert.assertFalse(aclManager.checkDAGModifyAccess(user1, groups1));
- Assert.assertFalse(aclManager.checkDAGModifyAccess(user2, groups2));
- Assert.assertTrue(aclManager.checkDAGModifyAccess(user3, noGroups));
- Assert.assertFalse(aclManager.checkDAGModifyAccess(user4, noGroups));
- Assert.assertTrue(aclManager.checkDAGModifyAccess(user5, groups3));
- Assert.assertTrue(aclManager.checkDAGModifyAccess(user6, noGroups));
+ UserGroupInformation dagUser = UserGroupInformation.createUserForTesting("dagUser", noGroups);
+
+ ACLManager amAclManager = new ACLManager(currentUser.getShortUserName(), conf);
+ ACLManager aclManager = new ACLManager(amAclManager, dagUser.getShortUserName(), conf);
+
+ Assert.assertTrue(aclManager.checkAMViewAccess(currentUser));
+ Assert.assertFalse(aclManager.checkAMViewAccess(dagUser));
+ Assert.assertTrue(aclManager.checkAMViewAccess(user1));
+ Assert.assertTrue(aclManager.checkAMViewAccess(user2));
+ Assert.assertFalse(aclManager.checkAMViewAccess(user3));
+ Assert.assertTrue(aclManager.checkAMViewAccess(user4));
+ Assert.assertFalse(aclManager.checkAMViewAccess(user5));
+ Assert.assertFalse(aclManager.checkAMViewAccess(user6));
+
+ Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(dagUser));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user1));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user2));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(user3));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user4));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(user5));
+ Assert.assertFalse(aclManager.checkAMModifyAccess(user6));
+
+ Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(dagUser));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user1));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user2));
+ Assert.assertFalse(aclManager.checkDAGViewAccess(user3));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user4));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user5));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(user6));
+
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(dagUser));
+ Assert.assertFalse(aclManager.checkDAGModifyAccess(user1));
+ Assert.assertFalse(aclManager.checkDAGModifyAccess(user2));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(user3));
+ Assert.assertFalse(aclManager.checkDAGModifyAccess(user4));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(user5));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(user6));
}
@@ -290,15 +294,18 @@ public class TestACLManager {
conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
- ACLManager aclManager = new ACLManager("a1", conf);
- Assert.assertTrue(aclManager.checkAMViewAccess("a1", null));
- Assert.assertTrue(aclManager.checkAMViewAccess("u1", null));
- Assert.assertTrue(aclManager.checkAMModifyAccess("a1", null));
- Assert.assertTrue(aclManager.checkAMModifyAccess("u1", null));
- Assert.assertTrue(aclManager.checkDAGViewAccess("a1", null));
- Assert.assertTrue(aclManager.checkDAGViewAccess("u1", null));
- Assert.assertTrue(aclManager.checkDAGModifyAccess("a1", null));
- Assert.assertTrue(aclManager.checkDAGModifyAccess("u1", null));
+ UserGroupInformation a1 = UserGroupInformation.createUserForTesting("a1", noGroups);
+ UserGroupInformation u1 = UserGroupInformation.createUserForTesting("u1", noGroups);
+
+ ACLManager aclManager = new ACLManager(a1.getShortUserName(), conf);
+ Assert.assertTrue(aclManager.checkAMViewAccess(a1));
+ Assert.assertTrue(aclManager.checkAMViewAccess(u1));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(a1));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(u1));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(a1));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(u1));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(a1));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(u1));
}
@Test
@@ -309,25 +316,29 @@ public class TestACLManager {
String modifyACLs = "a2,u2 ";
conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
- ACLManager aclManager = new ACLManager("a1", conf);
- Assert.assertTrue(aclManager.checkAMViewAccess("a1", null));
- Assert.assertTrue(aclManager.checkAMViewAccess("u1", null));
- Assert.assertTrue(aclManager.checkAMModifyAccess("a1", null));
- Assert.assertTrue(aclManager.checkAMModifyAccess("u1", null));
- Assert.assertTrue(aclManager.checkDAGViewAccess("a1", null));
- Assert.assertTrue(aclManager.checkDAGViewAccess("u1", null));
- Assert.assertTrue(aclManager.checkDAGModifyAccess("a1", null));
- Assert.assertTrue(aclManager.checkDAGModifyAccess("u1", null));
+
+ UserGroupInformation a1 = UserGroupInformation.createUserForTesting("a1", noGroups);
+ UserGroupInformation u1 = UserGroupInformation.createUserForTesting("u1", noGroups);
+
+ ACLManager aclManager = new ACLManager(a1.getShortUserName(), conf);
+ Assert.assertTrue(aclManager.checkAMViewAccess(a1));
+ Assert.assertTrue(aclManager.checkAMViewAccess(u1));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(a1));
+ Assert.assertTrue(aclManager.checkAMModifyAccess(u1));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(a1));
+ Assert.assertTrue(aclManager.checkDAGViewAccess(u1));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(a1));
+ Assert.assertTrue(aclManager.checkDAGModifyAccess(u1));
ACLManager dagAclManager = new ACLManager(aclManager, "dagUser", null);
- Assert.assertTrue(dagAclManager.checkAMViewAccess("a1", null));
- Assert.assertTrue(dagAclManager.checkAMViewAccess("u1", null));
- Assert.assertTrue(dagAclManager.checkAMModifyAccess("a1", null));
- Assert.assertTrue(dagAclManager.checkAMModifyAccess("u1", null));
- Assert.assertTrue(dagAclManager.checkDAGViewAccess("a1", null));
- Assert.assertTrue(dagAclManager.checkDAGViewAccess("u1", null));
- Assert.assertTrue(dagAclManager.checkDAGModifyAccess("a1", null));
- Assert.assertTrue(dagAclManager.checkDAGModifyAccess("u1", null));
+ Assert.assertTrue(dagAclManager.checkAMViewAccess(a1));
+ Assert.assertTrue(dagAclManager.checkAMViewAccess(u1));
+ Assert.assertTrue(dagAclManager.checkAMModifyAccess(a1));
+ Assert.assertTrue(dagAclManager.checkAMModifyAccess(u1));
+ Assert.assertTrue(dagAclManager.checkDAGViewAccess(a1));
+ Assert.assertTrue(dagAclManager.checkDAGViewAccess(u1));
+ Assert.assertTrue(dagAclManager.checkDAGModifyAccess(a1));
+ Assert.assertTrue(dagAclManager.checkDAGModifyAccess(u1));
}
@Test
http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
index 6381b71..c054305 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
@@ -62,17 +62,9 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
this.real = real;
}
- private String getRPCUserName() throws ServiceException {
+ private UserGroupInformation getRPCUser() throws ServiceException {
try {
- return UserGroupInformation.getCurrentUser().getShortUserName();
- } catch (IOException e) {
- throw wrapException(e);
- }
- }
-
- private List<String> getRPCUserGroups() throws ServiceException {
- try {
- return Arrays.asList(UserGroupInformation.getCurrentUser().getGroupNames());
+ return UserGroupInformation.getCurrentUser();
} catch (IOException e) {
throw wrapException(e);
}
@@ -81,8 +73,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
@Override
public GetAllDAGsResponseProto getAllDAGs(RpcController controller,
GetAllDAGsRequestProto request) throws ServiceException {
- String user = getRPCUserName();
- if (!real.getACLManager().checkAMViewAccess(user, getRPCUserGroups())) {
+ UserGroupInformation user = getRPCUser();
+ if (!real.getACLManager().checkAMViewAccess(user)) {
throw new AccessControlException("User " + user + " cannot perform AM view operation");
}
try{
@@ -96,10 +88,10 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
@Override
public GetDAGStatusResponseProto getDAGStatus(RpcController controller,
GetDAGStatusRequestProto request) throws ServiceException {
- String user = getRPCUserName();
+ UserGroupInformation user = getRPCUser();
try {
String dagId = request.getDagId();
- if (!real.getACLManager(dagId).checkDAGViewAccess(user, getRPCUserGroups())) {
+ if (!real.getACLManager(dagId).checkDAGViewAccess(user)) {
throw new AccessControlException("User " + user + " cannot perform DAG view operation");
}
DAGStatus status;
@@ -118,10 +110,10 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
@Override
public GetVertexStatusResponseProto getVertexStatus(RpcController controller,
GetVertexStatusRequestProto request) throws ServiceException {
- String user = getRPCUserName();
+ UserGroupInformation user = getRPCUser();
try {
String dagId = request.getDagId();
- if (!real.getACLManager(dagId).checkDAGViewAccess(user, getRPCUserGroups())) {
+ if (!real.getACLManager(dagId).checkDAGViewAccess(user)) {
throw new AccessControlException("User " + user + " cannot perform DAG view operation");
}
String vertexName = request.getVertexName();
@@ -140,10 +132,10 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
@Override
public TryKillDAGResponseProto tryKillDAG(RpcController controller,
TryKillDAGRequestProto request) throws ServiceException {
- String user = getRPCUserName();
+ UserGroupInformation user = getRPCUser();
try {
String dagId = request.getDagId();
- if (!real.getACLManager(dagId).checkDAGModifyAccess(user, getRPCUserGroups())) {
+ if (!real.getACLManager(dagId).checkDAGModifyAccess(user)) {
throw new AccessControlException("User " + user + " cannot perform DAG modify operation");
}
real.tryKillDAG(dagId);
@@ -156,8 +148,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
@Override
public SubmitDAGResponseProto submitDAG(RpcController controller,
SubmitDAGRequestProto request) throws ServiceException {
- String user = getRPCUserName();
- if (!real.getACLManager().checkAMModifyAccess(user, getRPCUserGroups())) {
+ UserGroupInformation user = getRPCUser();
+ if (!real.getACLManager().checkAMModifyAccess(user)) {
throw new AccessControlException("User " + user + " cannot perform AM modify operation");
}
try{
@@ -181,8 +173,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
@Override
public ShutdownSessionResponseProto shutdownSession(RpcController arg0,
ShutdownSessionRequestProto arg1) throws ServiceException {
- String user = getRPCUserName();
- if (!real.getACLManager().checkAMModifyAccess(user, getRPCUserGroups())) {
+ UserGroupInformation user = getRPCUser();
+ if (!real.getACLManager().checkAMModifyAccess(user)) {
throw new AccessControlException("User " + user + " cannot perform AM modify operation");
}
real.shutdownAM();
@@ -192,8 +184,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
@Override
public GetAMStatusResponseProto getAMStatus(RpcController controller,
GetAMStatusRequestProto request) throws ServiceException {
- String user = getRPCUserName();
- if (!real.getACLManager().checkAMViewAccess(user, getRPCUserGroups())) {
+ UserGroupInformation user = getRPCUser();
+ if (!real.getACLManager().checkAMViewAccess(user)) {
throw new AccessControlException("User " + user + " cannot perform AM view operation");
}
try {
[22/25] git commit: Update CHANGES.txt related to TEZ-1585
Posted by je...@apache.org.
Update CHANGES.txt related to TEZ-1585
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/3b34c418
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/3b34c418
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/3b34c418
Branch: refs/heads/TEZ-8
Commit: 3b34c4182839c63a0fa0dac66b22d8990acb04e0
Parents: 938097d
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Tue Sep 16 12:10:58 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Tue Sep 16 12:10:58 2014 +0530
----------------------------------------------------------------------
CHANGES.txt | 1 +
1 file changed, 1 insertion(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/3b34c418/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 97cf79b..ca4e7e2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -41,6 +41,7 @@ ALL CHANGES
TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
of DAG submission
TEZ-1571. Add create method for DataSinkDescriptor.
+ TEZ-1585. Memory leak in tez session mode.
Release 0.5.0: 2014-09-03
[04/25] git commit: TEZ-1543. Shuffle Errors on heavy load (causing
task retries) (Rajesh Balamohan)
Posted by je...@apache.org.
TEZ-1543. Shuffle Errors on heavy load (causing task retries) (Rajesh Balamohan)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/bf6ac4e7
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/bf6ac4e7
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/bf6ac4e7
Branch: refs/heads/TEZ-8
Commit: bf6ac4e7811c2ea5a327cac7d8a109e3c57fd8e7
Parents: 80b5795
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Fri Sep 12 03:57:40 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Sep 12 03:57:40 2014 +0530
----------------------------------------------------------------------
.../tez/runtime/library/shuffle/common/HttpConnection.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/bf6ac4e7/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
index 7208c16..d664f88 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
@@ -160,7 +160,7 @@ public class HttpConnection {
break;
} catch (IOException ioe) {
// Don't attempt another connect if already cleanedup.
- if (!cleanup) {
+ if (cleanup) {
LOG.info("Cleanup is set to true. Not attempting to"
+ " connect again. Last exception was: ["
+ ioe.getClass().getName() + ", " + ioe.getMessage() + "]");
@@ -170,7 +170,7 @@ public class HttpConnection {
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) {
+ if (connectionTimeout <= 0) {
throw ioe;
}
// reset the connect timeout for the last try
[05/25] git commit: TEZ-1494. DAG hangs waiting for
ShuffleManager.getNextInput() (Rajesh Balamohan)
Posted by je...@apache.org.
TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput() (Rajesh Balamohan)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/fcc74261
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/fcc74261
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/fcc74261
Branch: refs/heads/TEZ-8
Commit: fcc74261a0d76d00580f4d0dca042a1ed014ccec
Parents: bf6ac4e
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Fri Sep 12 04:09:21 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Sep 12 04:09:21 2014 +0530
----------------------------------------------------------------------
.../dag/impl/ImmediateStartVertexManager.java | 92 ++++-
.../app/dag/impl/RootInputVertexManager.java | 27 +-
.../tez/dag/app/dag/impl/TestVertexImpl.java | 364 +++++++++++++++++--
3 files changed, 417 insertions(+), 66 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/fcc74261/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
index b202d70..ac2b851 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
@@ -18,9 +18,11 @@
package org.apache.tez.dag.app.dag.impl;
-import java.util.List;
-import java.util.Map;
-
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.api.EdgeProperty;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.VertexManagerPlugin;
import org.apache.tez.dag.api.VertexManagerPluginContext;
@@ -28,29 +30,101 @@ import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
import org.apache.tez.runtime.api.Event;
import org.apache.tez.runtime.api.events.VertexManagerEvent;
-import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
/**
* Starts all tasks immediately on vertex start
*/
public class ImmediateStartVertexManager extends VertexManagerPlugin {
+ private static final Log LOG = LogFactory.getLog(ImmediateStartVertexManager.class);
+
+ private final Map<String, SourceVertexInfo> srcVertexInfo = Maps.newHashMap();
+ private int managedTasks;
+ private boolean tasksScheduled = false;
+
+ class SourceVertexInfo {
+ EdgeProperty edgeProperty;
+ int numFinishedTasks;
+
+ SourceVertexInfo(EdgeProperty edgeProperty) {
+ this.edgeProperty = edgeProperty;
+ }
+ }
+
public ImmediateStartVertexManager(VertexManagerPluginContext context) {
super(context);
}
@Override
public void onVertexStarted(Map<String, List<Integer>> completions) {
- int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
- List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
- for (int i=0; i<numTasks; ++i) {
- scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
+ managedTasks = getContext().getVertexNumTasks(getContext().getVertexName());
+ Map<String, EdgeProperty> edges = getContext().getInputVertexEdgeProperties();
+ for (Map.Entry<String, EdgeProperty> entry : edges.entrySet()) {
+ String srcVertex = entry.getKey();
+ EdgeProperty edgeProp = entry.getValue();
+ srcVertexInfo.put(srcVertex, new SourceVertexInfo(edgeProp));
+ }
+
+ //handle completions
+ for (Map.Entry<String, List<Integer>> entry : completions.entrySet()) {
+ for (Integer task : entry.getValue()) {
+ handleSourceTaskFinished(entry.getKey(), task);
+ }
+ }
+ scheduleTasks();
+ }
+
+ private void handleSourceTaskFinished(String vertex, Integer taskId) {
+ SourceVertexInfo srcInfo = srcVertexInfo.get(vertex);
+ //Not mandatory to check for duplicate completions here
+ srcInfo.numFinishedTasks++;
+ }
+
+ private void scheduleTasks() {
+ if (!canScheduleTasks()) {
+ return;
+ }
+
+ List<TaskWithLocationHint> tasksToStart = Lists.newArrayListWithCapacity(managedTasks);
+ for (int i = 0; i < managedTasks; ++i) {
+ tasksToStart.add(new TaskWithLocationHint(new Integer(i), null));
+ }
+
+ if (!tasksToStart.isEmpty()) {
+ LOG.info("Starting " + tasksToStart.size() + " in " + getContext().getVertexName());
+ getContext().scheduleVertexTasks(tasksToStart);
}
- getContext().scheduleVertexTasks(scheduledTasks);
+ tasksScheduled = true;
+ }
+
+ private boolean canScheduleTasks() {
+ //Check if at least 1 task is finished from each source vertex (in case of broadcast &
+ // one-to-one or custom)
+ for (Map.Entry<String, SourceVertexInfo> entry : srcVertexInfo.entrySet()) {
+ SourceVertexInfo srcVertexInfo = entry.getValue();
+ switch(srcVertexInfo.edgeProperty.getDataMovementType()) {
+ case ONE_TO_ONE:
+ case BROADCAST:
+ case CUSTOM:
+ if (srcVertexInfo.numFinishedTasks == 0) {
+ //do not schedule tasks until a task from source task is complete
+ return false;
+ }
+ default:
+ break;
+ }
+ }
+ return true;
}
@Override
public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
+ handleSourceTaskFinished(srcVertexName, attemptId);
+ if (!tasksScheduled) {
+ scheduleTasks();
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/fcc74261/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
index e6ffdc5..e850286 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
@@ -18,27 +18,23 @@
package org.apache.tez.dag.app.dag.impl;
-import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.UserPayload;
-import org.apache.tez.dag.api.VertexManagerPlugin;
import org.apache.tez.dag.api.VertexManagerPluginContext;
-import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
import org.apache.tez.runtime.api.Event;
import org.apache.tez.runtime.api.InputSpecUpdate;
import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
import org.apache.tez.runtime.api.events.InputDataInformationEvent;
import org.apache.tez.runtime.api.events.InputUpdatePayloadEvent;
-import org.apache.tez.runtime.api.events.VertexManagerEvent;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
-public class RootInputVertexManager extends VertexManagerPlugin {
+public class RootInputVertexManager extends ImmediateStartVertexManager {
private String configuredInputName;
@@ -46,27 +42,6 @@ public class RootInputVertexManager extends VertexManagerPlugin {
super(context);
}
- @Override
- public void initialize() {
- }
-
- @Override
- public void onVertexStarted(Map<String, List<Integer>> completions) {
- int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
- List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
- for (int i=0; i<numTasks; ++i) {
- scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
- }
- getContext().scheduleVertexTasks(scheduledTasks);
- }
-
- @Override
- public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
- }
-
- @Override
- public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) {
- }
@Override
public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor,
http://git-wip-us.apache.org/repos/asf/tez/blob/fcc74261/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 d894928..04e2219 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
@@ -20,6 +20,7 @@ package org.apache.tez.dag.app.dag.impl;
import java.nio.ByteBuffer;
+import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Mockito.doReturn;
@@ -1879,39 +1880,39 @@ public class TestVertexImpl {
Assert.assertEquals(2, v3.getOutputVerticesCount());
Assert.assertEquals(2, v3.getOutputVerticesCount());
- Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
+ assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
.getSourceVertexName())
|| "vertex2".equals(v3.getInputSpecList(0).get(0)
- .getSourceVertexName()));
- Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
+ .getSourceVertexName()));
+ assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
.getSourceVertexName())
|| "vertex2".equals(v3.getInputSpecList(0).get(1)
- .getSourceVertexName()));
- Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
+ .getSourceVertexName()));
+ assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
.getInputDescriptor().getClassName())
|| "i3_v2".equals(v3.getInputSpecList(0).get(0)
- .getInputDescriptor().getClassName()));
- Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
+ .getInputDescriptor().getClassName()));
+ assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
.getInputDescriptor().getClassName())
|| "i3_v2".equals(v3.getInputSpecList(0).get(1)
- .getInputDescriptor().getClassName()));
+ .getInputDescriptor().getClassName()));
- Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
+ assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
.getDestinationVertexName())
|| "vertex5".equals(v3.getOutputSpecList(0).get(0)
- .getDestinationVertexName()));
- Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
+ .getDestinationVertexName()));
+ assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
.getDestinationVertexName())
|| "vertex5".equals(v3.getOutputSpecList(0).get(1)
- .getDestinationVertexName()));
- Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
+ .getDestinationVertexName()));
+ assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
.getOutputDescriptor().getClassName())
|| "o3_v5".equals(v3.getOutputSpecList(0).get(0)
- .getOutputDescriptor().getClassName()));
- Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
+ .getOutputDescriptor().getClassName()));
+ assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
.getOutputDescriptor().getClassName())
|| "o3_v5".equals(v3.getOutputSpecList(0).get(1)
- .getOutputDescriptor().getClassName()));
+ .getOutputDescriptor().getClassName()));
}
@Test(timeout = 5000)
@@ -1940,13 +1941,13 @@ public class TestVertexImpl {
Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
Collections.singletonMap(
v1.getName(), mockEdgeManagerDescriptor);
- Assert.assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
- Assert.assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
+ assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
+ assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
EdgeManagerForTest);
Assert.assertEquals(1, v3.getTotalTasks());
Assert.assertEquals(1, tasks.size());
// the last one is removed
- Assert.assertTrue(tasks.keySet().iterator().next().equals(firstTask));
+ assertTrue(tasks.keySet().iterator().next().equals(firstTask));
}
@@ -1993,7 +1994,7 @@ public class TestVertexImpl {
Edge edge = edges.get("e4");
EdgeManagerPlugin em = edge.getEdgeManager();
EdgeManagerForTest originalEm = (EdgeManagerForTest) em;
- Assert.assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
+ assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
.getUserPayload().deepCopyAsArray()));
UserPayload userPayload = UserPayload.create(ByteBuffer.wrap(new String("foo").getBytes()));
@@ -2007,7 +2008,7 @@ public class TestVertexImpl {
Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
Collections.singletonMap(v3.getName(), edgeManagerDescriptor);
- Assert.assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
+ assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
edgeManagerDescriptors, null)); // Must decrease.
VertexImpl v5Impl = (VertexImpl) v5;
@@ -2015,10 +2016,10 @@ public class TestVertexImpl {
EdgeManagerPlugin modifiedEdgeManager = v5Impl.sourceVertices.get(v3)
.getEdgeManager();
Assert.assertNotNull(modifiedEdgeManager);
- Assert.assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
+ assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
// Ensure initialize() is called with the correct payload
- Assert.assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
+ assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
((EdgeManagerForTest) modifiedEdgeManager).getUserPayload().deepCopyAsArray()));
}
@@ -2092,7 +2093,7 @@ public class TestVertexImpl {
Assert.assertEquals(VertexTerminationCause.OWN_TASK_FAILURE, v.getTerminationCause());
String diagnostics =
StringUtils.join(v.getDiagnostics(), ",").toLowerCase();
- Assert.assertTrue(diagnostics.contains("task failed"
+ assertTrue(diagnostics.contains("task failed"
+ ", taskid=" + t1.toString()));
}
@@ -2104,7 +2105,7 @@ public class TestVertexImpl {
String diagnostics =
StringUtils.join(v2.getDiagnostics(), ",").toLowerCase();
LOG.info("diagnostics v2: " + diagnostics);
- Assert.assertTrue(diagnostics.contains(
+ assertTrue(diagnostics.contains(
"vertex received kill in inited state"));
}
@@ -2118,7 +2119,7 @@ public class TestVertexImpl {
killVertex(v3);
String diagnostics =
StringUtils.join(v3.getDiagnostics(), ",").toLowerCase();
- Assert.assertTrue(diagnostics.contains(
+ assertTrue(diagnostics.contains(
"vertex received kill while in running state"));
}
@@ -2200,7 +2201,7 @@ public class TestVertexImpl {
Assert.assertNull(v2.getOutputCommitter("output"));
VertexImpl v6 = vertices.get("vertex6");
- Assert.assertTrue(v6.getOutputCommitter("outputx")
+ assertTrue(v6.getOutputCommitter("outputx")
instanceof CountingOutputCommitter);
}
@@ -2208,11 +2209,11 @@ public class TestVertexImpl {
public void testVertexManagerInit() {
initAllVertices(VertexState.INITED);
VertexImpl v2 = vertices.get("vertex2");
- Assert.assertTrue(v2.getVertexManager().getPlugin()
+ assertTrue(v2.getVertexManager().getPlugin()
instanceof ImmediateStartVertexManager);
VertexImpl v6 = vertices.get("vertex6");
- Assert.assertTrue(v6.getVertexManager().getPlugin()
+ assertTrue(v6.getVertexManager().getPlugin()
instanceof ShuffleVertexManager);
}
@@ -3020,6 +3021,306 @@ public class TestVertexImpl {
}
}
+ @Test(timeout = 5000)
+ /**
+ * Ref: TEZ-1494
+ * If broadcast, one-to-one or custom edges are present in source, tasks should not start until
+ * 1 task from each source vertex is complete.
+ */
+ public void testTaskSchedulingWithCustomEdges() {
+ setupPreDagCreation();
+ dagPlan = createCustomDAGWithCustomEdges();
+ setupPostDagCreation();
+
+ /**
+ *
+ * M2 --(SG)--> R3 --(B)--\
+ * \
+ * M7 --(B)---------------->M5 ---(SG)--> R6
+ * /
+ * M8---(C)--------------->/
+ */
+
+ //init M2
+ VertexImpl m2 = vertices.get("M2");
+ VertexImpl m7 = vertices.get("M7");
+ VertexImpl r3 = vertices.get("R3");
+ VertexImpl m5 = vertices.get("M5");
+ VertexImpl m8 = vertices.get("M8");
+
+ initVertex(m2);
+ initVertex(m7);
+ initVertex(m8);
+ assertTrue(m7.getState().equals(VertexState.INITED));
+ assertTrue(m5.getState().equals(VertexState.INITED));
+ assertTrue(m8.getState().equals(VertexState.INITED));
+ assertTrue(m7.getVertexManager().getPlugin() instanceof ImmediateStartVertexManager);
+
+ //Start M2; Let tasks complete in M2; Also let 1 task complete in R3
+ dispatcher.getEventHandler().handle(new VertexEvent(m2.getVertexId(), VertexEventType.V_START));
+ dispatcher.await();
+ VertexEventTaskAttemptCompleted taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+ (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m2.getVertexId(),0), 0), TaskAttemptStateInternal.SUCCEEDED);
+ VertexEventTaskCompleted taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m2
+ .getVertexId(), 0), TaskState.SUCCEEDED);
+ dispatcher.getEventHandler().handle(taskAttemptCompleted);
+ dispatcher.getEventHandler().handle(taskCompleted);
+ dispatcher.await();
+ taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+ (TezTaskAttemptID.getInstance(TezTaskID.getInstance(r3.getVertexId(),0), 0),
+ TaskAttemptStateInternal.SUCCEEDED);
+ taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(r3
+ .getVertexId(), 0), TaskState.SUCCEEDED);
+ dispatcher.getEventHandler().handle(taskAttemptCompleted);
+ dispatcher.getEventHandler().handle(taskCompleted);
+ dispatcher.await();
+ assertTrue(m2.getState().equals(VertexState.SUCCEEDED));
+ assertTrue(m5.numSuccessSourceAttemptCompletions == 1);
+ assertTrue(m5.getState().equals(VertexState.INITED));
+
+ //R3 should be in running state as it has one task completed, and rest are pending
+ assertTrue(r3.getState().equals(VertexState.RUNNING));
+
+ //Let us start M7; M5 should start not start as it is dependent on M8 as well
+ dispatcher.getEventHandler().handle(new VertexEvent(m7.getVertexId(),VertexEventType.V_START));
+ dispatcher.await();
+ //Let one of the tasks get over in M7 as well.
+ taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+ (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m7.getVertexId(),0), 0),
+ TaskAttemptStateInternal.SUCCEEDED);
+ taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m7
+ .getVertexId(), 0), TaskState.SUCCEEDED);
+ dispatcher.getEventHandler().handle(taskAttemptCompleted);
+ dispatcher.getEventHandler().handle(taskCompleted);
+ dispatcher.await();
+ assertTrue(m5.numSuccessSourceAttemptCompletions == 2);
+
+ //M5 should be in INITED state, as it depends on M8
+ assertTrue(m5.getState().equals(VertexState.INITED));
+ for(Task task : m5.getTasks().values()) {
+ assertTrue(task.getState().equals(TaskState.NEW));
+ }
+
+ //Let us start M8; M5 should start now
+ dispatcher.getEventHandler().handle(new VertexEvent(m8.getVertexId(),VertexEventType.V_START));
+ dispatcher.await();
+
+ //M5 in running state. But tasks should not be scheduled until M8 finishes a task.
+ assertTrue(m5.getState().equals(VertexState.RUNNING));
+ for(Task task : m5.getTasks().values()) {
+ assertTrue(task.getState().equals(TaskState.NEW));
+ }
+
+ //Let one of the tasks get over in M8 as well. This should trigger tasks to be scheduled in M5
+ taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+ (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m8.getVertexId(),0), 0),
+ TaskAttemptStateInternal.SUCCEEDED);
+ taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m8
+ .getVertexId(), 0), TaskState.SUCCEEDED);
+ dispatcher.getEventHandler().handle(taskAttemptCompleted);
+ dispatcher.getEventHandler().handle(taskCompleted);
+ dispatcher.await();
+
+ assertTrue(m5.numSuccessSourceAttemptCompletions == 3);
+ //Ensure all tasks in M5 are in scheduled state
+ for(Task task : m5.getTasks().values()) {
+ assertTrue(task.getState().equals(TaskState.SCHEDULED));
+ }
+ }
+
+ //For TEZ-1494
+ private DAGPlan createCustomDAGWithCustomEdges() {
+ /**
+ *
+ * M2 --(SG)--> R3 --(B)--\
+ * \
+ * M7 --(B)---------------->M5 ---(SG)--> R6
+ * /
+ * M8---(C)--------------->/
+ */
+ DAGPlan dag = DAGPlan.newBuilder().setName("TestSamplerDAG")
+ .addVertex(VertexPlan.newBuilder()
+ .setName("M2")
+ .setProcessorDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
+ .setTaskConfig(PlanTaskConfiguration.newBuilder()
+ .setNumTasks(1)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("M2.class")
+ .build()
+ )
+ .addOutEdgeId("M2_R3")
+ .build()
+ )
+ .addVertex(VertexPlan.newBuilder()
+ .setName("M8")
+ .setProcessorDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName("M8.class"))
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
+ .setTaskConfig(PlanTaskConfiguration.newBuilder()
+ .setNumTasks(1)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("M8.class")
+ .build()
+ )
+ .addOutEdgeId("M8_M5")
+ .build()
+ )
+ .addVertex(VertexPlan.newBuilder()
+ .setName("R3")
+ .setProcessorDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host2").addRack("rack1").build())
+ .setTaskConfig(PlanTaskConfiguration.newBuilder()
+ .setNumTasks(10)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("R3.class")
+ .build()
+ )
+ .addInEdgeId("M2_R3")
+ .addOutEdgeId("R3_M5")
+ .build()
+ )
+ .addVertex(VertexPlan.newBuilder()
+ .setName("M5")
+ .setProcessorDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName("M5.class"))
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
+ .setTaskConfig(PlanTaskConfiguration.newBuilder()
+ .setNumTasks(10)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("M5.class")
+ .build()
+ )
+ .addInEdgeId("R3_M5")
+ .addInEdgeId("M7_M5")
+ .addInEdgeId("M8_M5")
+ .addOutEdgeId("M5_R6")
+ .build()
+ )
+ .addVertex(VertexPlan.newBuilder()
+ .setName("M7")
+ .setProcessorDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName("M7.class"))
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host4").addRack("rack1").build())
+ .setTaskConfig(PlanTaskConfiguration.newBuilder()
+ .setNumTasks(10)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("M7.class")
+ .build()
+ )
+ .addOutEdgeId("M7_M5")
+ .build()
+ )
+ .addVertex(VertexPlan.newBuilder()
+ .setName("R6")
+ .setProcessorDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName("R6.class"))
+ .setType(PlanVertexType.NORMAL)
+ .addTaskLocationHint(
+ PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
+ .setTaskConfig(PlanTaskConfiguration.newBuilder()
+ .setNumTasks(1)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("R6.class")
+ .build()
+ )
+ .addInEdgeId("M5_R6")
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3"))
+ .setInputVertexName("M2")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3.class"))
+ .setOutputVertexName("R3")
+ .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
+ .setId("M2_R3")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5"))
+ .setInputVertexName("R3")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5.class"))
+ .setOutputVertexName("M5")
+ .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+ .setId("R3_M5")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5"))
+ .setInputVertexName("M7")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5.class"))
+ .setOutputVertexName("M5")
+ .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+ .setId("M7_M5")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6"))
+ .setInputVertexName("M5")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6.class"))
+ .setOutputVertexName("R6")
+ .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
+ .setId("M5_R6")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5"))
+ .setInputVertexName("M8")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5.class"))
+ .setEdgeManager(
+ TezEntityDescriptorProto.newBuilder()
+ .setClassName(EdgeManagerForTest.class.getName())
+ .setUserPayload(ByteString.copyFrom(edgePayload))
+ .build())
+ .setOutputVertexName("M5")
+ .setDataMovementType(PlanEdgeDataMovementType.CUSTOM)
+ .setId("M8_M5")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .build();
+
+ return dag;
+ }
+
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
public void testVertexWithInitializerSuccess() {
@@ -3215,6 +3516,7 @@ public class TestVertexImpl {
dispatcher.getEventHandler(), taskAttemptListener,
clock, thh, true, appContext, vertexLocationHint, null, taskSpecificLaunchCmdOption,
updateTracker);
+ v.setInputVertices(new HashMap());
vertexIdMap.put(vId, v);
vertices.put(v.getName(), v);
v.handle(new VertexEvent(vId, VertexEventType.V_INIT));
@@ -3454,8 +3756,8 @@ public class TestVertexImpl {
List<GroupInputSpec> groupInSpec = vC.getGroupInputSpecList(0);
Assert.assertEquals(1, groupInSpec.size());
Assert.assertEquals("Group", groupInSpec.get(0).getGroupName());
- Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
- Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
+ assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
+ assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
groupInSpec.get(0).getMergedInputDescriptor().getClassName().equals("Group.class");
}
[12/25] git commit: TEZ-1575. MRRSleepJob does not pick MR settings
for container size and java opts. (hitesh)
Posted by je...@apache.org.
TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts. (hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b62298e1
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b62298e1
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b62298e1
Branch: refs/heads/TEZ-8
Commit: b62298e1054c934e4ee8909b7ba3165ce344ea8b
Parents: d1ea44d
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 11:02:51 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 11:02:51 2014 -0700
----------------------------------------------------------------------
.../tez/mapreduce/examples/MRRSleepJob.java | 25 +++++++++++++-------
1 file changed, 16 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b62298e1/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 3f9d222..de8a1ad 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -525,9 +525,11 @@ public class MRRSleepJob extends Configured implements Tool {
int numTasks = generateSplitsInAM ? -1 : numMapper;
Vertex mapVertex = Vertex.create("map", ProcessorDescriptor.create(
- MapProcessor.class.getName()).setUserPayload(mapUserPayload), numTasks)
- .addTaskLocalFiles(commonLocalResources);
- mapVertex.addDataSource("MRInput", dataSource);
+ MapProcessor.class.getName()).setUserPayload(mapUserPayload), numTasks,
+ MRHelpers.getResourceForMRMapper(mapStageConf));
+ mapVertex.addTaskLocalFiles(commonLocalResources)
+ .addDataSource("MRInput", dataSource)
+ .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
vertices.add(mapVertex);
if (iReduceStagesCount > 0
@@ -538,8 +540,11 @@ public class MRRSleepJob extends Configured implements Tool {
UserPayload iReduceUserPayload = TezUtils.createUserPayloadFromConf(iconf);
Vertex ivertex = Vertex.create("ireduce" + (i + 1),
ProcessorDescriptor.create(ReduceProcessor.class.getName()).
- setUserPayload(iReduceUserPayload), numIReducer);
- ivertex.addTaskLocalFiles(commonLocalResources);
+ setUserPayload(iReduceUserPayload), numIReducer,
+ MRHelpers.getResourceForMRReducer(intermediateReduceStageConfs[i]));
+ ivertex.addTaskLocalFiles(commonLocalResources)
+ .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(
+ intermediateReduceStageConfs[i]));
vertices.add(ivertex);
}
}
@@ -548,10 +553,12 @@ public class MRRSleepJob extends Configured implements Tool {
if (numReducer > 0) {
UserPayload reducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
finalReduceVertex = Vertex.create("reduce", ProcessorDescriptor.create(
- ReduceProcessor.class.getName()).setUserPayload(reducePayload), numReducer);
- finalReduceVertex.addTaskLocalFiles(commonLocalResources);
- finalReduceVertex.addDataSink("MROutput", MROutputLegacy.createConfigBuilder(finalReduceConf,
- NullOutputFormat.class).build());
+ ReduceProcessor.class.getName()).setUserPayload(reducePayload), numReducer,
+ MRHelpers.getResourceForMRReducer(finalReduceConf));
+ finalReduceVertex.addTaskLocalFiles(commonLocalResources)
+ .addDataSink("MROutput", MROutputLegacy.createConfigBuilder(
+ finalReduceConf, NullOutputFormat.class).build())
+ .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
vertices.add(finalReduceVertex);
} else {
// Map only job
[02/25] git commit: TEZ-850. Recovery unit tests. (Jeff Zhang via
hitesh)
Posted by je...@apache.org.
TEZ-850. Recovery unit tests. (Jeff Zhang via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/f65e65ae
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/f65e65ae
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/f65e65ae
Branch: refs/heads/TEZ-8
Commit: f65e65aea8cbdb44dd65c6590fbe38dd84413a5a
Parents: d6589d3
Author: Hitesh Shah <hi...@apache.org>
Authored: Thu Sep 11 13:22:21 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Thu Sep 11 13:22:21 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/tez/dag/app/dag/impl/DAGImpl.java | 45 +-
.../tez/dag/app/dag/impl/TaskAttemptImpl.java | 3 +-
.../apache/tez/dag/app/dag/impl/TaskImpl.java | 12 +-
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 240 +++---
.../tez/dag/app/dag/impl/TestDAGImpl.java | 6 +-
.../tez/dag/app/dag/impl/TestDAGRecovery.java | 514 +++++++++++
.../app/dag/impl/TestTaskAttemptRecovery.java | 178 ++++
.../tez/dag/app/dag/impl/TestTaskRecovery.java | 629 +++++++++++---
.../dag/app/dag/impl/TestVertexRecovery.java | 860 +++++++++++++++++++
10 files changed, 2243 insertions(+), 245 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 032438f..4fc7e83 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
ALL CHANGES:
TEZ-1544. Link to release artifacts for 0.5.0 does not point to a specific link for 0.5.0.
TEZ-1559. Add system tests for AM recovery.
+ TEZ-850. Recovery unit tests.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 24d2e6b..daaa81b 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
@@ -139,7 +139,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
private final TaskAttemptListener taskAttemptListener;
private final TaskHeartbeatHandler taskHeartbeatHandler;
private final Object tasksSyncHandle = new Object();
-
+
private volatile boolean committedOrAborted = false;
private volatile boolean allOutputsCommitted = false;
boolean commitAllOutputsOnSuccess = true;
@@ -157,7 +157,8 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
private final StateChangeNotifier entityUpdateTracker;
volatile Map<TezVertexID, Vertex> vertices = new HashMap<TezVertexID, Vertex>();
- private Map<String, Edge> edges = new HashMap<String, Edge>();
+ @VisibleForTesting
+ Map<String, Edge> edges = new HashMap<String, Edge>();
private TezCounters dagCounters = new TezCounters();
private Object fullCountersLock = new Object();
private TezCounters fullCounters = null;
@@ -359,14 +360,18 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
private DAGTerminationCause terminationCause;
private Credentials credentials;
- private long initTime;
- private long startTime;
- private long finishTime;
-
+ @VisibleForTesting
+ long initTime;
+ @VisibleForTesting
+ long startTime;
+ @VisibleForTesting
+ long finishTime;
+
Map<String, VertexGroupInfo> vertexGroups = Maps.newHashMap();
Map<String, List<VertexGroupInfo>> vertexGroupInfo = Maps.newHashMap();
private DAGState recoveredState = DAGState.NEW;
- private boolean recoveryCommitInProgress = false;
+ @VisibleForTesting
+ boolean recoveryCommitInProgress = false;
Map<String, Boolean> recoveredGroupCommits = new HashMap<String, Boolean>();
static class VertexGroupInfo {
@@ -381,7 +386,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
groupMembers = Sets.newHashSet(groupInfo.getGroupMembersList());
edgeMergedInputs = Maps.newHashMapWithExpectedSize(groupInfo.getEdgeMergedInputsCount());
for (PlanGroupInputEdgeInfo edgInfo : groupInfo.getEdgeMergedInputsList()) {
- edgeMergedInputs.put(edgInfo.getDestVertexName(),
+ edgeMergedInputs.put(edgInfo.getDestVertexName(),
DagTypeConverters.convertInputDescriptorFromDAGPlan(edgInfo.getMergedInput()));
}
outputs = Sets.newHashSet(groupInfo.getOutputsList());
@@ -706,7 +711,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
}
}
-
+
private boolean commitOutput(String outputName, OutputCommitter outputCommitter) {
final OutputCommitter committer = outputCommitter;
try {
@@ -723,7 +728,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
return false;
}
-
+
private synchronized boolean commitOrAbortOutputs(boolean dagSucceeded) {
if (this.committedOrAborted) {
LOG.info("Ignoring multiple output commit/abort");
@@ -731,7 +736,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
LOG.info("Calling DAG commit/abort for dag: " + getID());
this.committedOrAborted = true;
-
+
boolean successfulOutputsAlreadyCommitted = !commitAllOutputsOnSuccess;
boolean failedWhileCommitting = false;
if (dagSucceeded && !successfulOutputsAlreadyCommitted) {
@@ -772,7 +777,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
LOG.info("No output committers for vertex: " + vertex.getName());
continue;
}
- Map<String, OutputCommitter> outputCommitters =
+ Map<String, OutputCommitter> outputCommitters =
new HashMap<String, OutputCommitter>(vertex.getOutputCommitters());
Set<String> sharedOutputs = vertex.getSharedOutputs();
// remove shared outputs
@@ -793,7 +798,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
LOG.info("Committing output: " + entry.getKey() + " for vertex: "
+ vertex.getVertexId());
if (vertex.getState() != VertexState.SUCCEEDED) {
- throw new TezUncheckedException("Vertex: " + vertex.getName() +
+ throw new TezUncheckedException("Vertex: " + vertex.getName() +
" not in SUCCEEDED state. State= " + vertex.getState());
}
if (!commitOutput(entry.getKey(), entry.getValue())) {
@@ -803,11 +808,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
}
}
-
+
if (failedWhileCommitting) {
LOG.info("DAG: " + getID() + " failed while committing");
}
-
+
if (!dagSucceeded || failedWhileCommitting) {
// come here because dag failed or
// dag succeeded and all or none semantics were on and a commit failed
@@ -1026,9 +1031,9 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
if (finishTime == 0) {
setFinishTime();
}
-
+
boolean allOutputsCommitted = commitOrAbortOutputs(finalState == DAGState.SUCCEEDED);
-
+
if (finalState == DAGState.SUCCEEDED && !allOutputsCommitted) {
finalState = DAGState.FAILED;
trySetTerminationCause(DAGTerminationCause.COMMIT_FAILURE);
@@ -1057,7 +1062,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
LOG.info("DAG: " + getID() + " finished with state: " + finalState);
return finalState;
}
-
+
private DAGStatus.State getDAGStatusFromState(DAGState finalState) {
switch (finalState) {
case NEW:
@@ -1631,7 +1636,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
+ ", numFailedVertices=" + job.numFailedVertices
+ ", numKilledVertices=" + job.numKilledVertices
+ ", numVertices=" + job.numVertices);
-
+
if (failed) {
return DAGState.TERMINATING;
}
@@ -1724,7 +1729,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
addDiagnostic("Vertex re-running"
+ ", vertexName=" + vertex.getName()
+ ", vertexId=" + vertex.getVertexId());
-
+
if (!commitAllOutputsOnSuccess) {
// partial output may already have been committed. fail if so
List<VertexGroupInfo> groupList = vertexGroupInfo.get(vertex.getName());
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 e63dbf5..7ba90b5 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
@@ -137,7 +137,8 @@ public class TaskAttemptImpl implements TaskAttempt,
private String nodeHttpAddress;
private String nodeRackName;
- private TaskAttemptStatus reportedStatus;
+ @VisibleForTesting
+ TaskAttemptStatus reportedStatus;
private DAGCounter localityCounter;
// Used to store locality information when
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 ecd2bcc..1dd711b 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
@@ -116,7 +116,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
protected final AppContext appContext;
private final Resource taskResource;
private final ContainerContext containerContext;
- private long scheduledTime;
+ @VisibleForTesting
+ long scheduledTime;
private final List<TezEvent> tezEventsForTaskAttempts = new ArrayList<TezEvent>();
private static final List<TezEvent> EMPTY_TASK_ATTEMPT_TEZ_EVENTS =
@@ -124,7 +125,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
// counts the number of attempts that are either running or in a state where
// they will come to be running when they get a Container
- private int numberUncompletedAttempts = 0;
+ @VisibleForTesting
+ int numberUncompletedAttempts = 0;
private boolean historyTaskStartGenerated = false;
@@ -290,11 +292,13 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
//saying COMMIT_PENDING
private TezTaskAttemptID commitAttempt;
- private TezTaskAttemptID successfulAttempt;
+ @VisibleForTesting
+ TezTaskAttemptID successfulAttempt;
@VisibleForTesting
int failedAttempts;
- private int finishedAttempts;//finish are total of success, failed and killed
+ @VisibleForTesting
+ int finishedAttempts;//finish are total of success, failed and killed
private final boolean leafVertex;
private TaskState recoveredState = TaskState.NEW;
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 ff556ba..31240cb 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
@@ -205,9 +205,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
//fields initialized in init
- private int numStartedSourceVertices = 0;
- private int numInitedSourceVertices = 0;
- private int numRecoveredSourceVertices = 0;
+ @VisibleForTesting
+ int numStartedSourceVertices = 0;
+ @VisibleForTesting
+ int numInitedSourceVertices = 0;
+ @VisibleForTesting
+ int numRecoveredSourceVertices = 0;
private int distanceFromRoot = 0;
@@ -238,7 +241,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
new VertexStateChangedCallback();
private VertexState recoveredState = VertexState.NEW;
- private List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
+ @VisibleForTesting
+ List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
private boolean vertexAlreadyInitialized = false;
protected static final
@@ -254,7 +258,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
VertexState.INITIALIZING, VertexState.FAILED),
VertexEventType.V_INIT,
new InitTransition())
- .addTransition(VertexState.NEW,
+ .addTransition(VertexState.NEW,
EnumSet.of(VertexState.NEW),
VertexEventType.V_NULL_EDGE_INITIALIZED,
new NullEdgeInitializedTransition())
@@ -308,7 +312,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
// Transitions from INITIALIZING state
.addTransition(VertexState.INITIALIZING,
- EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
+ EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
VertexState.FAILED),
VertexEventType.V_ROOT_INPUT_INITIALIZED,
new RootInputInitializedTransition())
@@ -341,14 +345,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
.addTransition(VertexState.INITIALIZING, VertexState.ERROR,
VertexEventType.V_INTERNAL_ERROR,
INTERNAL_ERROR_TRANSITION)
- .addTransition(VertexState.INITIALIZING,
+ .addTransition(VertexState.INITIALIZING,
EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
VertexState.FAILED),
VertexEventType.V_NULL_EDGE_INITIALIZED,
new NullEdgeInitializedTransition())
// Transitions from INITED state
- // SOURCE_VERTEX_STARTED - for sources which determine parallelism,
+ // SOURCE_VERTEX_STARTED - for sources which determine parallelism,
// they must complete before this vertex can start.
.addTransition
(VertexState.INITED,
@@ -358,14 +362,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
.addTransition(VertexState.INITED, VertexState.INITED,
VertexEventType.V_SOURCE_VERTEX_STARTED,
new SourceVertexStartedTransition())
- .addTransition(VertexState.INITED,
+ .addTransition(VertexState.INITED,
EnumSet.of(VertexState.INITED),
VertexEventType.V_ONE_TO_ONE_SOURCE_SPLIT,
new OneToOneSourceSplitTransition())
.addTransition(VertexState.INITED, VertexState.INITED,
VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED,
SOURCE_TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION)
- .addTransition(VertexState.INITED,
+ .addTransition(VertexState.INITED,
EnumSet.of(VertexState.RUNNING, VertexState.INITED),
VertexEventType.V_START,
new StartTransition())
@@ -393,7 +397,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
VertexState.ERROR),
VertexEventType.V_TASK_COMPLETED,
new TaskCompletedTransition())
- .addTransition(VertexState.RUNNING,
+ .addTransition(VertexState.RUNNING,
EnumSet.of(VertexState.RUNNING),
VertexEventType.V_ONE_TO_ONE_SOURCE_SPLIT,
new OneToOneSourceSplitTransition())
@@ -455,7 +459,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
VertexEventType.V_ROUTE_EVENT,
ROUTE_EVENT_TRANSITION)
.addTransition(
- VertexState.SUCCEEDED,
+ VertexState.SUCCEEDED,
EnumSet.of(VertexState.FAILED, VertexState.ERROR),
VertexEventType.V_TASK_COMPLETED,
new TaskCompletedAfterVertexSuccessTransition())
@@ -551,17 +555,27 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
private final StateMachineTez<VertexState, VertexEventType, VertexEvent, VertexImpl> stateMachine;
//changing fields while the vertex is running
- private int numTasks;
- private int completedTaskCount = 0;
- private int succeededTaskCount = 0;
- private int failedTaskCount = 0;
- private int killedTaskCount = 0;
-
- private long initTimeRequested; // Time at which INIT request was received.
- private long initedTime; // Time when entering state INITED
- private long startTimeRequested; // Time at which START request was received.
- private long startedTime; // Time when entering state STARTED
- private long finishTime;
+ @VisibleForTesting
+ int numTasks;
+ @VisibleForTesting
+ int completedTaskCount = 0;
+ @VisibleForTesting
+ int succeededTaskCount = 0;
+ @VisibleForTesting
+ int failedTaskCount = 0;
+ @VisibleForTesting
+ int killedTaskCount = 0;
+
+ @VisibleForTesting
+ long initTimeRequested; // Time at which INIT request was received.
+ @VisibleForTesting
+ long initedTime; // Time when entering state INITED
+ @VisibleForTesting
+ long startTimeRequested; // Time at which START request was received.
+ @VisibleForTesting
+ long startedTime; // Time when entering state STARTED
+ @VisibleForTesting
+ long finishTime;
private float progress;
private final TezVertexID vertexId; //runtime assigned id.
@@ -576,14 +590,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
private Map<Vertex, Edge> targetVertices;
Set<Edge> uninitializedEdges = Sets.newHashSet();
- private Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
+ private Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
rootInputDescriptors;
- private Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
+ private Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
additionalOutputs;
private Map<String, OutputCommitter> outputCommitters;
private Map<String, InputSpecUpdate> rootInputSpecs = new HashMap<String, InputSpecUpdate>();
private static final InputSpecUpdate DEFAULT_ROOT_INPUT_SPECS = InputSpecUpdate
- .getDefaultSinglePhysicalInputSpecUpdate();
+ .getDefaultSinglePhysicalInputSpecUpdate();
private final List<OutputSpec> additionalOutputSpecs = new ArrayList<OutputSpec>();
private Set<String> inputsWithInitializers;
private int numInitializedInputs;
@@ -598,7 +612,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
private RootInputInitializerManager rootInputInitializerManager;
VertexManager vertexManager;
-
+
private final UserGroupInformation dagUgi;
private boolean parallelismSet = false;
@@ -607,20 +621,22 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
private AtomicBoolean committed = new AtomicBoolean(false);
private AtomicBoolean aborted = new AtomicBoolean(false);
private boolean commitVertexOutputs = false;
-
+
private Map<String, VertexGroupInfo> dagVertexGroups;
-
+
private TaskLocationHint taskLocationHints[];
private Map<String, LocalResource> localResources;
private Map<String, String> environment;
private final String javaOpts;
private final ContainerContext containerContext;
private VertexTerminationCause terminationCause;
-
+
private String logIdentifier;
- private boolean recoveryCommitInProgress = false;
+ @VisibleForTesting
+ boolean recoveryCommitInProgress = false;
private boolean summaryCompleteSeen = false;
- private boolean hasCommitter = false;
+ @VisibleForTesting
+ boolean hasCommitter = false;
private boolean vertexCompleteSeen = false;
private Map<String,EdgeManagerPluginDescriptor> recoveredSourceEdgeManagers = null;
private Map<String, InputSpecUpdate> recoveredRootInputSpecUpdates = null;
@@ -658,7 +674,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
logLocationHints(this.vertexName, vertexLocationHint);
}
setTaskLocationHints(vertexLocationHint);
-
+
this.dagUgi = appContext.getCurrentDAG().getDagUGI();
this.taskResource = DagTypeConverters
@@ -897,12 +913,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
this.readLock.unlock();
}
}
-
+
@Override
public TaskLocationHint getTaskLocationHint(TezTaskID taskId) {
this.readLock.lock();
try {
- if (taskLocationHints == null ||
+ if (taskLocationHints == null ||
taskLocationHints.length <= taskId.getId()) {
return null;
}
@@ -1081,8 +1097,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
private void setTaskLocationHints(VertexLocationHint vertexLocationHint) {
- if (vertexLocationHint != null &&
- vertexLocationHint.getTaskLocationHints() != null &&
+ if (vertexLocationHint != null &&
+ vertexLocationHint.getTaskLocationHints() != null &&
!vertexLocationHint.getTaskLocationHints().isEmpty()) {
List<TaskLocationHint> locHints = vertexLocationHint.getTaskLocationHints();
taskLocationHints = locHints.toArray(new TaskLocationHint[locHints.size()]);
@@ -1154,7 +1170,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
}
}
-
+
// Restore any rootInputSpecUpdates which may have been registered during a parallelism
// update.
if (rootInputSpecUpdates != null) {
@@ -1166,7 +1182,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
writeLock.unlock();
}
}
- Preconditions.checkArgument(parallelism >= 0, "Parallelism must be >=0. Value: "
+ Preconditions.checkArgument(parallelism >= 0, "Parallelism must be >=0. Value: "
+ parallelism + " for vertex: " + logIdentifier);
setVertexLocationHint(vertexLocationHint);
writeLock.lock();
@@ -1175,7 +1191,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
LOG.info("Parallelism can only be set dynamically once per vertex: " + logIdentifier);
return false;
}
-
+
parallelismSet = true;
// Input initializer/Vertex Manager/1-1 split expected to set parallelism.
@@ -1185,7 +1201,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
"Vertex state is not Initializing. Value: " + getState()
+ " for vertex: " + logIdentifier);
}
-
+
if(sourceEdgeManagers != null) {
for(Map.Entry<String, EdgeManagerPluginDescriptor> entry : sourceEdgeManagers.entrySet()) {
LOG.info("Replacing edge manager for source:"
@@ -1223,7 +1239,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
stateChangeNotifier.stateChanged(vertexId,
new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
this.createTasks();
- LOG.info("Vertex " + getVertexId() +
+ LOG.info("Vertex " + getVertexId() +
" parallelism set to " + parallelism);
if (canInitVertex()) {
getEventHandler().handle(new VertexEvent(getVertexId(), VertexEventType.V_READY_TO_INIT));
@@ -1244,7 +1260,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
return false;
}
if (parallelism == numTasks) {
- LOG.info("setParallelism same as current value: " + parallelism +
+ LOG.info("setParallelism same as current value: " + parallelism +
" for vertex: " + logIdentifier);
Preconditions.checkArgument(sourceEdgeManagers != null,
"Source edge managers or RootInputSpecs must be set when not changing parallelism");
@@ -1258,7 +1274,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
for (Edge edge : sourceVertices.values()) {
edge.startEventBuffering();
}
-
+
// assign to local variable of LinkedHashMap to make sure that changing
// type of task causes compile error. We depend on LinkedHashMap for order
LinkedHashMap<TezTaskID, Task> currentTasks = this.tasks;
@@ -1281,14 +1297,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
LOG.info("Removing task: " + entry.getKey());
iter.remove();
}
- LOG.info("Vertex " + logIdentifier +
+ LOG.info("Vertex " + logIdentifier +
" parallelism set to " + parallelism + " from " + numTasks);
int oldNumTasks = numTasks;
this.numTasks = parallelism;
stateChangeNotifier.stateChanged(vertexId,
new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
assert tasks.size() == numTasks;
-
+
// set new edge managers
if(sourceEdgeManagers != null) {
for(Map.Entry<String, EdgeManagerPluginDescriptor> entry : sourceEdgeManagers.entrySet()) {
@@ -1320,17 +1336,17 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
edge.stopEventBuffering();
}
}
-
+
for (Map.Entry<Vertex, Edge> entry : targetVertices.entrySet()) {
Edge edge = entry.getValue();
- if (edge.getEdgeProperty().getDataMovementType()
+ if (edge.getEdgeProperty().getDataMovementType()
== DataMovementType.ONE_TO_ONE) {
// inform these target vertices that we have changed parallelism
- VertexEventOneToOneSourceSplit event =
+ VertexEventOneToOneSourceSplit event =
new VertexEventOneToOneSourceSplit(entry.getKey().getVertexId(),
getVertexId(),
- ((originalOneToOneSplitSource!=null) ?
- originalOneToOneSplitSource : getVertexId()),
+ ((originalOneToOneSplitSource!=null) ?
+ originalOneToOneSplitSource : getVertexId()),
numTasks);
getEventHandler().handle(event);
}
@@ -1339,7 +1355,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
} finally {
writeLock.unlock();
}
-
+
return true;
}
@@ -1733,7 +1749,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
// no code, for now
}
- private ContainerContext getContainerContext(int taskIdx) {
+ @VisibleForTesting
+ ContainerContext getContainerContext(int taskIdx) {
if (taskSpecificLaunchCmdOpts.addTaskSpecificLaunchCmdOption(vertexName, taskIdx)) {
String jvmOpts = taskSpecificLaunchCmdOpts.getTaskSpecificOption(javaOpts, vertexName, taskIdx);
ContainerContext context = new ContainerContext(this.localResources,
@@ -1806,9 +1823,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
if (rootInputDescriptors != null) {
LOG.info("Root Inputs exist for Vertex: " + getName() + " : "
+ rootInputDescriptors);
- for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input
+ for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input
: rootInputDescriptors.values()) {
- if (input.getControllerDescriptor() != null &&
+ if (input.getControllerDescriptor() != null &&
input.getControllerDescriptor().getClassName() != null) {
if (inputsWithInitializers == null) {
inputsWithInitializers = Sets.newHashSet();
@@ -1875,7 +1892,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
checkTaskLimits();
return VertexState.INITED;
}
-
+
private void assignVertexManager() {
boolean hasBipartite = false;
boolean hasOneToOne = false;
@@ -1895,12 +1912,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
hasCustom = true;
break;
default:
- throw new TezUncheckedException("Unknown data movement type: " +
+ throw new TezUncheckedException("Unknown data movement type: " +
edge.getEdgeProperty().getDataMovementType());
}
}
}
-
+
boolean hasUserVertexManager = vertexPlan.hasVertexManagerPlugin();
if (hasUserVertexManager) {
@@ -2008,6 +2025,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
vertex.finished(VertexState.ERROR);
}
+ // recover from recover log, should recover to running
+ // desiredState must be RUNNING based on above code
VertexState endState;
switch (vertex.recoveredState) {
case NEW:
@@ -2086,6 +2105,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
vertex.vertexManager.onVertexStarted(vertex.pendingReportedSrcCompletions);
endState = VertexState.RUNNING;
} else {
+ // why succeeded here
endState = VertexState.SUCCEEDED;
vertex.finished(endState);
}
@@ -2249,7 +2269,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
(vertex.sourceVertices == null || vertex.sourceVertices.containsKey(otherVertex) ||
vertex.targetVertices == null || vertex.targetVertices.containsKey(otherVertex)),
"Not connected to vertex " + otherVertex.getName() + " from vertex: " + vertex.logIdentifier);
- LOG.info("Edge initialized for connection to vertex " + otherVertex.getName() +
+ LOG.info("Edge initialized for connection to vertex " + otherVertex.getName() +
" at vertex : " + vertex.logIdentifier);
vertex.uninitializedEdges.remove(edge);
if(vertex.getState() == VertexState.INITIALIZING && vertex.canInitVertex()) {
@@ -2596,7 +2616,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
if (state.equals(VertexState.FAILED)) {
return state;
}
- // TODO move before to handle NEW state
+ // TODO move before to handle NEW state
if (vertex.targetVertices != null) {
for (Edge e : vertex.targetVertices.values()) {
if (e.getEdgeManager() == null) {
@@ -2619,7 +2639,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
}
}
-
+
// Create tasks based on initial configuration, but don't start them yet.
if (vertex.numTasks == -1) {
LOG.info("Num tasks is -1. Expecting VertexManager/InputInitializers/1-1 split"
@@ -2633,7 +2653,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
vertex.appContext.getTaskScheduler().getNumClusterNodes(),
vertex.getTaskResource(),
vertex.appContext.getTaskScheduler().getTotalResources());
- List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
+ List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
inputList = Lists.newArrayListWithCapacity(vertex.inputsWithInitializers.size());
for (String inputName : vertex.inputsWithInitializers) {
inputList.add(vertex.rootInputDescriptors.get(inputName));
@@ -2646,7 +2666,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
} else {
boolean hasOneToOneUninitedSource = false;
for (Map.Entry<Vertex, Edge> entry : vertex.sourceVertices.entrySet()) {
- if (entry.getValue().getEdgeProperty().getDataMovementType() ==
+ if (entry.getValue().getEdgeProperty().getDataMovementType() ==
DataMovementType.ONE_TO_ONE) {
if (entry.getKey().getTotalTasks() == -1) {
hasOneToOneUninitedSource = true;
@@ -2662,7 +2682,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
LOG.info("Vertex will initialize via custom vertex manager. " + vertex.logIdentifier);
return VertexState.INITIALIZING;
}
- throw new TezUncheckedException(vertex.getVertexId() +
+ throw new TezUncheckedException(vertex.getVertexId() +
" has -1 tasks but does not have input initializers, " +
"1-1 uninited sources or custom vertex manager to set it at runtime");
}
@@ -2676,14 +2696,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
vertex.appContext.getTaskScheduler().getNumClusterNodes(),
vertex.getTaskResource(),
vertex.appContext.getTaskScheduler().getTotalResources());
- List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
+ List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
inputList = Lists.newArrayListWithCapacity(vertex.inputsWithInitializers.size());
for (String inputName : vertex.inputsWithInitializers) {
inputList.add(vertex.rootInputDescriptors.get(inputName));
}
LOG.info("Starting root input initializers: "
+ vertex.inputsWithInitializers.size());
- // special case when numTasks>0 and still we want to stay in initializing
+ // special case when numTasks>0 and still we want to stay in initializing
// state. This is handled in RootInputInitializedTransition specially.
vertex.initWaitsForRootInitializers = true;
vertex.rootInputInitializerManager.runInputInitializers(inputList);
@@ -2711,7 +2731,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
Resource vertexTaskResource, Resource totalResource) {
return new RootInputInitializerManager(this, appContext, this.dagUgi, this.stateChangeNotifier);
}
-
+
private boolean initializeVertexInInitializingState() {
boolean isInitialized = initializeVertex();
if (!isInitialized) {
@@ -2721,7 +2741,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
return true;
}
-
+
void startIfPossible() {
if (startSignalPending) {
// Trigger a start event to ensure route events are seen before
@@ -2735,7 +2755,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
public static class VertexInitializedTransition implements
MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-
+
static VertexState doTransition(VertexImpl vertex) {
Preconditions.checkState(vertex.canInitVertex(), "Vertex: " + vertex.logIdentifier);
boolean isInitialized = vertex.initializeVertexInInitializingState();
@@ -2744,15 +2764,15 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
vertex.startIfPossible();
- return VertexState.INITED;
+ return VertexState.INITED;
}
-
+
@Override
public VertexState transition(VertexImpl vertex, VertexEvent event) {
return doTransition(vertex);
}
}
-
+
// present in most transitions so that the initializer thread can be shutdown properly
public static class RootInputInitializedTransition implements
MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
@@ -2773,15 +2793,15 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
// All inputs initialized, shutdown the initializer.
vertex.rootInputInitializerManager.shutdown();
}
-
+
// done. check if we need to do the initialization
- if (vertex.getState() == VertexState.INITIALIZING &&
+ if (vertex.getState() == VertexState.INITIALIZING &&
vertex.initWaitsForRootInitializers) {
// set the wait flag to false
vertex.initWaitsForRootInitializers = false;
// initialize vertex if possible and needed
if (vertex.canInitVertex()) {
- Preconditions.checkState(vertex.numTasks >= 0,
+ Preconditions.checkState(vertex.numTasks >= 0,
"Parallelism should have been set by now for vertex: " + vertex.logIdentifier);
return VertexInitializedTransition.doTransition(vertex);
}
@@ -2795,10 +2815,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
@Override
public VertexState transition(VertexImpl vertex, VertexEvent event) {
- VertexEventOneToOneSourceSplit splitEvent =
+ VertexEventOneToOneSourceSplit splitEvent =
(VertexEventOneToOneSourceSplit)event;
TezVertexID originalSplitSource = splitEvent.getOriginalSplitSource();
-
+
if (vertex.originalOneToOneSplitSource != null) {
VertexState state = vertex.getState();
Preconditions
@@ -2813,25 +2833,25 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
if (vertex.originalOneToOneSplitSource.equals(originalSplitSource)) {
// ignore another split event that may have come from a different
// path in the DAG. We have already split because of that source
- LOG.info("Ignoring split of vertex " + vertex.getVertexId() +
- " because of split in vertex " + originalSplitSource +
+ LOG.info("Ignoring split of vertex " + vertex.getVertexId() +
+ " because of split in vertex " + originalSplitSource +
" sent by vertex " + splitEvent.getSenderVertex() +
" numTasks " + splitEvent.getNumTasks());
return state;
}
// cannot split from multiple sources
- throw new TezUncheckedException("Vertex: " + vertex.getVertexId() +
- " asked to split by: " + originalSplitSource +
+ throw new TezUncheckedException("Vertex: " + vertex.getVertexId() +
+ " asked to split by: " + originalSplitSource +
" but was already split by:" + vertex.originalOneToOneSplitSource);
}
-
- LOG.info("Splitting vertex " + vertex.getVertexId() +
- " because of split in vertex " + originalSplitSource +
+
+ LOG.info("Splitting vertex " + vertex.getVertexId() +
+ " because of split in vertex " + originalSplitSource +
" sent by vertex " + splitEvent.getSenderVertex() +
" numTasks " + splitEvent.getNumTasks());
vertex.originalOneToOneSplitSource = originalSplitSource;
vertex.setParallelism(splitEvent.getNumTasks(), null, null, null);
- if (vertex.getState() == VertexState.RUNNING ||
+ if (vertex.getState() == VertexState.RUNNING ||
vertex.getState() == VertexState.INITED) {
return vertex.getState();
} else {
@@ -2861,19 +2881,19 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
vertex.numStartedSourceVertices++;
LOG.info("Source vertex started: " + startEvent.getSourceVertexId() +
- " for vertex: " + vertex.getVertexId() + " numStartedSources: " +
+ " for vertex: " + vertex.getVertexId() + " numStartedSources: " +
vertex.numStartedSourceVertices + " numSources: " + vertex.sourceVertices.size());
-
+
if (vertex.numStartedSourceVertices < vertex.sourceVertices.size()) {
LOG.info("Cannot start vertex: " + vertex.logIdentifier + " numStartedSources: "
+ vertex.numStartedSourceVertices + " numSources: " + vertex.sourceVertices.size());
return;
}
-
- // vertex meets external start dependency conditions. Save this signal in
+
+ // vertex meets external start dependency conditions. Save this signal in
// case we are not ready to start now and need to start later
vertex.startSignalPending = true;
-
+
if (vertex.getState() != VertexState.INITED) {
// vertex itself is not ready to start. External dependencies have already
// notified us.
@@ -2883,14 +2903,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
+ vertex.uninitializedEdges.size());
return;
}
-
+
// vertex is inited and all dependencies are ready. Inited vertex means
// parallelism must be set already and edges defined
Preconditions.checkState(
(vertex.numTasks >= 0 && vertex.uninitializedEdges.isEmpty()),
"Cannot start vertex that is not completely defined. Vertex: "
+ vertex.logIdentifier + " numTasks: " + vertex.numTasks);
-
+
vertex.startIfPossible();
}
}
@@ -2906,14 +2926,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
+ " initWaitsForRootInitializers: " + initWaitsForRootInitializers);
return false;
}
-
- public static class StartWhileInitializingTransition implements
+
+ public static class StartWhileInitializingTransition implements
SingleArcTransition<VertexImpl, VertexEvent> {
@Override
public void transition(VertexImpl vertex, VertexEvent event) {
// vertex state machine does not start itself in the initializing state
- // this start event can only come directly from the DAG. That means this
+ // this start event can only come directly from the DAG. That means this
// is a top level vertex of the dag
Preconditions.checkState(
(vertex.sourceVertices == null || vertex.sourceVertices.isEmpty()),
@@ -2926,10 +2946,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
public static class StartTransition implements
MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-
+
@Override
public VertexState transition(VertexImpl vertex, VertexEvent event) {
- Preconditions.checkState(vertex.getState() == VertexState.INITED,
+ Preconditions.checkState(vertex.getState() == VertexState.INITED,
"Unexpected state " + vertex.getState() + " for " + vertex.logIdentifier);
vertex.startTimeRequested = vertex.clock.getTime();
return vertex.startVertex();
@@ -2937,7 +2957,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
private VertexState startVertex() {
- Preconditions.checkState(getState() == VertexState.INITED,
+ Preconditions.checkState(getState() == VertexState.INITED,
"Vertex must be inited " + logIdentifier);
startedTime = clock.getTime();
@@ -3095,7 +3115,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
case OWN_TASK_FAILURE: vertex.tryEnactKill(trigger, TaskTerminationCause.OTHER_TASK_FAILURE); break;
case ROOT_INPUT_INIT_FAILURE:
case COMMIT_FAILURE:
- case INVALID_NUM_OF_TASKS:
+ case INVALID_NUM_OF_TASKS:
case INIT_FAILURE:
case INTERNAL_ERROR:
case OTHER_VERTEX_FAILURE: vertex.tryEnactKill(trigger, TaskTerminationCause.OTHER_VERTEX_FAILURE); break;
@@ -3172,7 +3192,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
if (taskEvent.getState() == TaskState.SUCCEEDED) {
taskSucceeded(vertex, task);
} else if (taskEvent.getState() == TaskState.FAILED) {
- LOG.info("Failing vertex: " + vertex.logIdentifier +
+ LOG.info("Failing vertex: " + vertex.logIdentifier +
" because task failed: " + taskEvent.getTaskID());
vertex.tryEnactKill(VertexTerminationCause.OWN_TASK_FAILURE, TaskTerminationCause.OTHER_TASK_FAILURE);
forceTransitionToKillWait = true;
@@ -3220,7 +3240,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
vertex.succeededTaskCount--;
}
}
-
+
private static class VertexNoTasksCompletedTransition implements
MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
@@ -3229,7 +3249,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
return VertexImpl.checkVertexForCompletion(vertex);
}
}
-
+
private static class TaskCompletedAfterVertexSuccessTransition implements
MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
@Override
@@ -3241,12 +3261,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
if (vEvent.getState() == TaskState.FAILED) {
finalState = VertexState.FAILED;
finalStatus = VertexStatus.State.FAILED;
- diagnosticMsg = "Vertex " + vertex.logIdentifier +" failed as task " + vEvent.getTaskID() +
+ diagnosticMsg = "Vertex " + vertex.logIdentifier +" failed as task " + vEvent.getTaskID() +
" failed after vertex succeeded.";
} else {
finalState = VertexState.ERROR;
finalStatus = VertexStatus.State.ERROR;
- diagnosticMsg = "Vertex " + vertex.logIdentifier + " error as task " + vEvent.getTaskID() +
+ diagnosticMsg = "Vertex " + vertex.logIdentifier + " error as task " + vEvent.getTaskID() +
" completed with state " + vEvent.getState() + " after vertex succeeded.";
}
LOG.info(diagnosticMsg);
@@ -3599,14 +3619,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
@Nullable
@Override
- public Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
+ public Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
getAdditionalInputs() {
return this.rootInputDescriptors;
}
@Nullable
@Override
- public Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
+ public Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
getAdditionalOutputs() {
return this.additionalOutputs;
}
@@ -3696,7 +3716,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
inputSpecList = new ArrayList<InputSpec>(this.getInputVerticesCount()
+ (rootInputDescriptors == null ? 0 : rootInputDescriptors.size()));
if (rootInputDescriptors != null) {
- for (Entry<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
+ for (Entry<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
rootInputDescriptorEntry : rootInputDescriptors.entrySet()) {
inputSpecList.add(new InputSpec(rootInputDescriptorEntry.getKey(),
rootInputDescriptorEntry.getValue().getIODescriptor(), rootInputSpecs.get(
@@ -3729,18 +3749,18 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
return outputSpecList;
}
-
+
//TODO Eventually remove synchronization.
@Override
public synchronized List<GroupInputSpec> getGroupInputSpecList(int taskIndex) {
return groupInputSpecList;
}
-
+
@Override
public synchronized void addSharedOutputs(Set<String> outputs) {
this.sharedOutputs.addAll(outputs);
}
-
+
@Override
public synchronized Set<String> getSharedOutputs() {
return this.sharedOutputs;
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 ec05815..aba4fd9 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
@@ -111,7 +111,7 @@ public class TestDAGImpl {
private static final Log LOG = LogFactory.getLog(TestDAGImpl.class);
private DAGPlan dagPlan;
private TezDAGID dagId;
- private Configuration conf;
+ private static Configuration conf;
private DrainDispatcher dispatcher;
private Credentials fsTokens;
private AppContext appContext;
@@ -344,7 +344,7 @@ public class TestDAGImpl {
}
// Create a plan with 3 vertices: A, B, C. Group(A,B)->C
- private DAGPlan createGroupDAGPlan() {
+ static DAGPlan createGroupDAGPlan() {
LOG.info("Setting up group dag plan");
int dummyTaskCount = 1;
Resource dummyTaskResource = Resource.newInstance(1, 1);
@@ -381,7 +381,7 @@ public class TestDAGImpl {
return dag.createDag(conf);
}
- private DAGPlan createTestDAGPlan() {
+ public static DAGPlan createTestDAGPlan() {
LOG.info("Setting up dag plan");
DAGPlan dag = DAGPlan.newBuilder()
.setName("testverteximpl")
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
new file mode 100644
index 0000000..da0186e
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
@@ -0,0 +1,514 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.app.dag.impl;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.DAGState;
+import org.apache.tez.dag.app.dag.VertexState;
+import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDAGFinished;
+import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventRecoverEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventType;
+import org.apache.tez.dag.app.dag.event.VertexEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
+import org.apache.tez.dag.history.events.DAGCommitStartedEvent;
+import org.apache.tez.dag.history.events.DAGFinishedEvent;
+import org.apache.tez.dag.history.events.DAGInitializedEvent;
+import org.apache.tez.dag.history.events.DAGStartedEvent;
+import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
+import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
+import org.apache.tez.dag.records.TezDAGID;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class TestDAGRecovery {
+
+ private DAGImpl dag;
+ private EventHandler mockEventHandler;
+
+ private String user = "root";
+ private String dagName = "dag1";
+
+ private AppContext mockAppContext;
+ private ApplicationId appId = ApplicationId.newInstance(
+ System.currentTimeMillis(), 1);
+ private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+ private long initTime = 100L;
+ private long startTime = initTime + 200L;
+ private long commitStartTime = startTime + 200L;
+ private long finishTime = commitStartTime + 200L;
+
+ @Before
+ public void setUp() {
+
+ mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+ when(mockAppContext.getCurrentDAG().getDagUGI()).thenReturn(null);
+ mockEventHandler = mock(EventHandler.class);
+
+ DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+ dag =
+ new DAGImpl(dagId, new Configuration(), dagPlan, mockEventHandler,
+ mock(TaskAttemptListener.class), new Credentials(),
+ new SystemClock(), user, mock(TaskHeartbeatHandler.class),
+ mockAppContext);
+ }
+
+ private void assertNewState() {
+ assertEquals(0, dag.getVertices().size());
+ assertEquals(0, dag.edges.size());
+ assertNull(dag.dagScheduler);
+ assertFalse(dag.recoveryCommitInProgress);
+ assertEquals(0, dag.recoveredGroupCommits.size());
+ }
+
+ private void restoreFromDAGInitializedEvent() {
+ DAGState recoveredState =
+ dag.restoreFromEvent(new DAGInitializedEvent(dagId, initTime, user,
+ dagName));
+ assertEquals(DAGState.INITED, recoveredState);
+ assertEquals(initTime, dag.initTime);
+ assertEquals(6, dag.getVertices().size());
+ assertEquals(6, dag.edges.size());
+ assertNotNull(dag.dagScheduler);
+ }
+
+ private void restoreFromDAGStartedEvent() {
+ DAGState recoveredState =
+ dag.restoreFromEvent(new DAGStartedEvent(dagId, startTime, user,
+ dagName));
+ assertEquals(startTime, dag.startTime);
+ assertEquals(DAGState.RUNNING, recoveredState);
+ }
+
+ private void restoreFromDAGCommitStartedEvent() {
+ DAGState recoveredState =
+ dag.restoreFromEvent(new DAGCommitStartedEvent(dagId, commitStartTime));
+ assertTrue(dag.recoveryCommitInProgress);
+ assertEquals(DAGState.RUNNING, recoveredState);
+ }
+
+ private void restoreFromVertexGroupCommitStartedEvent() {
+ DAGState recoveredState =
+ dag.restoreFromEvent(new VertexGroupCommitStartedEvent(dagId, "g1",
+ commitStartTime));
+ assertEquals(1, dag.recoveredGroupCommits.size());
+ assertFalse(dag.recoveredGroupCommits.get("g1").booleanValue());
+ assertEquals(DAGState.RUNNING, recoveredState);
+ }
+
+ private void restoreFromVertexGroupCommitFinishedEvent() {
+ DAGState recoveredState =
+ dag.restoreFromEvent(new VertexGroupCommitFinishedEvent(dagId, "g1",
+ commitStartTime + 100L));
+ assertEquals(1, dag.recoveredGroupCommits.size());
+ assertTrue(dag.recoveredGroupCommits.get("g1").booleanValue());
+ assertEquals(DAGState.RUNNING, recoveredState);
+ }
+
+ private void restoreFromDAGFinishedEvent(DAGState finalState) {
+ DAGState recoveredState =
+ dag.restoreFromEvent(new DAGFinishedEvent(dagId, startTime, finishTime,
+ finalState, "", new TezCounters(), user, dagName));
+ assertEquals(finishTime, dag.finishTime);
+ assertFalse(dag.recoveryCommitInProgress);
+ assertEquals(finalState, recoveredState);
+ }
+
+ /**
+ * New -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_FromNew() {
+ assertNewState();
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+ ArgumentCaptor<DAGEvent> eventCaptor =
+ ArgumentCaptor.forClass(DAGEvent.class);
+ verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+ List<DAGEvent> events = eventCaptor.getAllValues();
+ assertEquals(2, events.size());
+ assertEquals(DAGEventType.DAG_INIT, events.get(0).getType());
+ assertEquals(DAGEventType.DAG_START, events.get(1).getType());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_FromInited() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+ assertEquals(DAGState.RUNNING, dag.getState());
+ // send recover event to 2 root vertex
+ ArgumentCaptor<VertexEvent> eventCaptor =
+ ArgumentCaptor.forClass(VertexEvent.class);
+ verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+ List<VertexEvent> vertexEvents = eventCaptor.getAllValues();
+ assertEquals(2, vertexEvents.size());
+ for (VertexEvent vEvent : vertexEvents) {
+ assertTrue(vEvent instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vEvent;
+ assertEquals(VertexState.RUNNING, recoverEvent.getDesiredState());
+ }
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_FromStarted() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+ assertEquals(DAGState.RUNNING, dag.getState());
+ // send recover event to 2 root vertex
+ ArgumentCaptor<VertexEvent> eventCaptor =
+ ArgumentCaptor.forClass(VertexEvent.class);
+ verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+ List<VertexEvent> vertexEvents = eventCaptor.getAllValues();
+ assertEquals(2, vertexEvents.size());
+ for (VertexEvent vEvent : vertexEvents) {
+ assertTrue(vEvent instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vEvent;
+ assertEquals(VertexState.RUNNING, recoverEvent.getDesiredState());
+ }
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * restoreFromDAGFinishedEvent (SUCCEEDED) -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_Finished_SUCCEEDED() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+ restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.SUCCEEDED, dag.getState());
+
+ // recover all the vertices to SUCCEED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * restoreFromDAGFinishedEvent(FAILED) -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_Finished_FAILED() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+ restoreFromDAGFinishedEvent(DAGState.FAILED);
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.FAILED, dag.getState());
+
+ // recover all the vertices to FAILED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.FAILED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
+ * restoreFromDAGFinishedEvent -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_Finished_KILLED() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+ restoreFromDAGFinishedEvent(DAGState.KILLED);
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.KILLED, dag.getState());
+
+ // recover all the vertices to KILLED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.KILLED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.KILLED, dagFinishedEvent.getDAGState());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
+ * restoreFromDAGFinishedEvent -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_Finished_ERROR() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+ restoreFromDAGFinishedEvent(DAGState.ERROR);
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.ERROR, dag.getState());
+
+ // recover all the vertices to KILLED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.FAILED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.ERROR, dagFinishedEvent.getDAGState());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * restoreFromDAG_COMMIT_STARTED -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_COMMIT_STARTED() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+ restoreFromDAGCommitStartedEvent();
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.FAILED, dag.getState());
+
+ // recover all the vertices to SUCCEEDED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * restoreFromDAG_COMMIT_STARTED -> -> restoreFromDAGFinished (SUCCEEDED)->
+ * RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_COMMIT_STARTED_Finished_SUCCEEDED() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+
+ restoreFromDAGCommitStartedEvent();
+ restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.SUCCEEDED, dag.getState());
+
+ // recover all the vertices to SUCCEED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * restoreFromVERTEX_GROUP_COMMIT_STARTED -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_GROUP_COMMIT_STARTED() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+ restoreFromVertexGroupCommitStartedEvent();
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.FAILED, dag.getState());
+
+ // recover all the vertices to SUCCEEDED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
+ * RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_GROUP_COMMIT_STARTED_FINISHED() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+
+ restoreFromVertexGroupCommitStartedEvent();
+ restoreFromVertexGroupCommitFinishedEvent();
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+ assertEquals(DAGState.RUNNING, dag.getState());
+
+ // send recover event to 2 root vertex
+ verify(mockEventHandler, times(2)).handle(
+ any(VertexEventRecoverVertex.class));
+ assertEquals(DAGState.RUNNING, dag.getState());
+ }
+
+ /**
+ * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+ * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
+ * restoreFromDAG_Finished -> RecoverTransition
+ */
+ @Test
+ public void testDAGRecovery_GROUP_COMMIT_Finished() {
+ assertNewState();
+ restoreFromDAGInitializedEvent();
+ restoreFromDAGStartedEvent();
+
+ restoreFromVertexGroupCommitStartedEvent();
+ restoreFromVertexGroupCommitFinishedEvent();
+ restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+
+ dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+ // recover all the vertices to SUCCEEDED
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ int i = 0;
+ for (; i < 6; ++i) {
+ assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+ VertexEventRecoverVertex recoverEvent =
+ (VertexEventRecoverVertex) events.get(i);
+ assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+ }
+
+ // send DAGAppMasterEventDAGFinished at last
+ assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+ DAGAppMasterEventDAGFinished dagFinishedEvent =
+ (DAGAppMasterEventDAGFinished) events.get(i);
+ assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
new file mode 100644
index 0000000..3b04cf6
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.app.dag.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
+import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.junit.Before;
+import org.junit.Test;
+
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class TestTaskAttemptRecovery {
+
+ private TaskAttemptImpl ta;
+ private EventHandler mockEventHandler;
+ private long startTime = System.currentTimeMillis();
+ private long finishTime = startTime + 5000;
+
+ private TezTaskAttemptID taId = mock(TezTaskAttemptID.class);
+ private String vertexName = "v1";
+
+ @Before
+ public void setUp() {
+ mockEventHandler = mock(EventHandler.class);
+ TezTaskID taskId =
+ TezTaskID.fromString("task_1407371892933_0001_1_00_000000");
+ ta =
+ new TaskAttemptImpl(taskId, 0, mockEventHandler,
+ mock(TaskAttemptListener.class), new Configuration(),
+ new SystemClock(), mock(TaskHeartbeatHandler.class),
+ mock(AppContext.class), false, Resource.newInstance(1, 1),
+ mock(ContainerContext.class), false);
+ }
+
+ private void restoreFromTAStartEvent() {
+ TaskAttemptState recoveredState =
+ ta.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
+ startTime, mock(ContainerId.class), mock(NodeId.class), "", ""));
+ assertEquals(startTime, ta.getLaunchTime());
+ assertEquals(TaskAttemptState.RUNNING, recoveredState);
+ }
+
+ private void restoreFromTAFinishedEvent(TaskAttemptState state) {
+ String diag = "test_diag";
+ TezCounters counters = mock(TezCounters.class);
+
+ TaskAttemptState recoveredState =
+ ta.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+ startTime, finishTime, state, diag, counters));
+ assertEquals(startTime, ta.getLaunchTime());
+ assertEquals(finishTime, ta.getFinishTime());
+ assertEquals(counters, ta.reportedStatus.counters);
+ assertEquals(1.0f, ta.reportedStatus.progress, 1e-6);
+ assertEquals(state, ta.reportedStatus.state);
+ assertEquals(1, ta.getDiagnostics().size());
+ assertEquals(diag, ta.getDiagnostics().get(0));
+ assertEquals(state, recoveredState);
+ }
+
+ /**
+ * No any event to restore -> RecoverTransition
+ */
+ @Test
+ public void testTARecovery_NEW() {
+ ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+ assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+ verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+ }
+
+ /**
+ * restoreFromTAStartEvent -> RecoverTransition
+ */
+ @Test
+ public void testTARecovery_START() {
+ restoreFromTAStartEvent();
+
+ ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+ assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+ verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+ }
+
+ /**
+ * restoreFromTAStartEvent -> restoreFromTAFinished (SUCCEED)
+ * -> RecoverTransition
+ */
+ @Test
+ public void testTARecovery_SUCCEED() {
+ restoreFromTAStartEvent();
+ restoreFromTAFinishedEvent(TaskAttemptState.SUCCEEDED);
+
+ ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+ assertEquals(TaskAttemptStateInternal.SUCCEEDED, ta.getInternalState());
+ verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+ }
+
+ /**
+ * restoreFromTAStartEvent -> restoreFromTAFinished (KILLED)
+ * -> RecoverTransition
+ */
+ @Test
+ public void testTARecovery_KIILED() {
+ restoreFromTAStartEvent();
+ restoreFromTAFinishedEvent(TaskAttemptState.KILLED);
+
+ ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+ assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+ verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+ }
+
+ /**
+ * restoreFromTAStartEvent -> restoreFromTAFinished (FAILED)
+ * -> RecoverTransition
+ */
+ @Test
+ public void testTARecovery_FAILED() {
+ restoreFromTAStartEvent();
+ restoreFromTAFinishedEvent(TaskAttemptState.FAILED);
+
+ ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+ assertEquals(TaskAttemptStateInternal.FAILED, ta.getInternalState());
+ verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+ }
+
+ /**
+ * restoreFromTAFinishedEvent ( no TAStartEvent before TAFinishedEvent )
+ */
+ @Test
+ public void testRecover_FINISH_BUT_NO_START() {
+ try {
+ restoreFromTAFinishedEvent(TaskAttemptState.SUCCEEDED);
+ fail("Should fail due to no TaskAttemptStartEvent before TaskAttemptFinishedEvent");
+ } catch (Throwable e) {
+ assertEquals("Finished Event seen but"
+ + " no Started Event was encountered earlier", e.getMessage());
+ }
+ }
+}
[06/25] git commit: TEZ-853. Support counters recovery. (Jeff Zhang
via hitesh)
Posted by je...@apache.org.
TEZ-853. Support counters recovery. (Jeff Zhang via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/16a0f579
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/16a0f579
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/16a0f579
Branch: refs/heads/TEZ-8
Commit: 16a0f57952d554e95ae1346ac3f360998ac7faac
Parents: fcc7426
Author: Hitesh Shah <hi...@apache.org>
Authored: Thu Sep 11 22:03:26 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Thu Sep 11 22:03:26 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/tez/common/counters/DAGCounter.java | 1 +
.../apache/tez/dag/api/client/DAGStatus.java | 2 +-
.../tez/dag/app/dag/impl/TaskAttemptImpl.java | 53 +++++------
.../dag/history/events/TaskFinishedEvent.java | 10 +--
.../dag/history/events/VertexFinishedEvent.java | 8 --
tez-dag/src/main/proto/HistoryEvents.proto | 4 +-
.../tez/dag/app/dag/impl/TestTaskAttempt.java | 16 ++++
.../app/dag/impl/TestTaskAttemptRecovery.java | 60 +++++++++++--
.../TestHistoryEventsProtoConversion.java | 6 --
.../org/apache/tez/test/TestAMRecovery.java | 94 +++++++++++++++++---
11 files changed, 178 insertions(+), 77 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4fc7e83..519aaa6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -10,6 +10,7 @@ ALL CHANGES:
TEZ-1544. Link to release artifacts for 0.5.0 does not point to a specific link for 0.5.0.
TEZ-1559. Add system tests for AM recovery.
TEZ-850. Recovery unit tests.
+ TEZ-853. Support counters recovery.
Release 0.5.1: Unreleased
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
index 3598572..b5bdffb 100644
--- a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
public enum DAGCounter {
NUM_FAILED_TASKS,
NUM_KILLED_TASKS,
+ NUM_SUCCEEDED_TASKS,
TOTAL_LAUNCHED_TASKS,
OTHER_LOCAL_TASKS,
DATA_LOCAL_TASKS,
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
index 297c6f2..7c51c42 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
@@ -170,7 +170,7 @@ public class DAGStatus {
+ ", diagnostics="
+ StringUtils.join(getDiagnostics(), LINE_SEPARATOR)
+ ", counters="
- + (dagCounters == null ? "null" : dagCounters.toString()));
+ + (getDAGCounters() == null ? "null" : getDAGCounters().toString()));
return sb.toString();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/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 7ba90b5..b8430cc 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
@@ -791,6 +791,7 @@ public class TaskAttemptImpl implements TaskAttempt,
this.launchTime = tEvent.getStartTime();
recoveryStartEventSeen = true;
recoveredState = TaskAttemptState.RUNNING;
+ sendEvent(createDAGCounterUpdateEventTALaunched(this));
return recoveredState;
}
case TASK_ATTEMPT_FINISHED:
@@ -806,6 +807,7 @@ public class TaskAttemptImpl implements TaskAttempt,
this.reportedStatus.state = tEvent.getState();
this.diagnostics.add(tEvent.getDiagnostics());
this.recoveredState = tEvent.getState();
+ sendEvent(createDAGCounterUpdateEventTAFinished(this, tEvent.getState()));
return recoveredState;
}
default:
@@ -829,47 +831,29 @@ public class TaskAttemptImpl implements TaskAttempt,
}
// TOOD Merge some of these JobCounter events.
- private static DAGEventCounterUpdate createJobCounterUpdateEventTALaunched(
+ private static DAGEventCounterUpdate createDAGCounterUpdateEventTALaunched(
TaskAttemptImpl ta) {
- DAGEventCounterUpdate jce =
+ DAGEventCounterUpdate dagCounterEvent =
new DAGEventCounterUpdate(
ta.getDAGID()
);
- jce.addCounterUpdate(DAGCounter.TOTAL_LAUNCHED_TASKS, 1);
- return jce;
+ dagCounterEvent.addCounterUpdate(DAGCounter.TOTAL_LAUNCHED_TASKS, 1);
+ return dagCounterEvent;
}
- private static DAGEventCounterUpdate createJobCounterUpdateEventSlotMillis(
- TaskAttemptImpl ta) {
+ private static DAGEventCounterUpdate createDAGCounterUpdateEventTAFinished(
+ TaskAttemptImpl taskAttempt, TaskAttemptState taState) {
DAGEventCounterUpdate jce =
- new DAGEventCounterUpdate(
- ta.getDAGID()
- );
+ new DAGEventCounterUpdate(taskAttempt.getDAGID());
-// long slotMillis = computeSlotMillis(ta);
-// jce.addCounterUpdate(DAGCounter.SLOTS_MILLIS_TASKS, slotMillis);
- return jce;
- }
-
- private static DAGEventCounterUpdate createJobCounterUpdateEventTATerminated(
- TaskAttemptImpl taskAttempt, boolean taskAlreadyCompleted,
- TaskAttemptStateInternal taState) {
- DAGEventCounterUpdate jce =
- new DAGEventCounterUpdate(
- taskAttempt.getDAGID());
-
- if (taState == TaskAttemptStateInternal.FAILED) {
+ if (taState == TaskAttemptState.FAILED) {
jce.addCounterUpdate(DAGCounter.NUM_FAILED_TASKS, 1);
- } else if (taState == TaskAttemptStateInternal.KILLED) {
+ } else if (taState == TaskAttemptState.KILLED) {
jce.addCounterUpdate(DAGCounter.NUM_KILLED_TASKS, 1);
+ } else if (taState == TaskAttemptState.SUCCEEDED ) {
+ jce.addCounterUpdate(DAGCounter.NUM_SUCCEEDED_TASKS, 1);
}
-// long slotMillisIncrement = computeSlotMillis(taskAttempt);
-// if (!taskAlreadyCompleted) {
-// // dont double count the elapsed time
-// jce.addCounterUpdate(DAGCounter.SLOTS_MILLIS_TASKS, slotMillisIncrement);
-// }
-
return jce;
}
@@ -1142,8 +1126,8 @@ public class TaskAttemptImpl implements TaskAttempt,
ta.addDiagnosticInfo(((DiagnosableEvent) event).getDiagnosticInfo());
}
- ta.sendEvent(createJobCounterUpdateEventTATerminated(ta, false,
- helper.getTaskAttemptStateInternal()));
+ ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
+ helper.getTaskAttemptState()));
if (ta.getLaunchTime() != 0) {
// TODO For cases like this, recovery goes for a toss, since the the
// attempt will not exist in the history file.
@@ -1184,7 +1168,7 @@ public class TaskAttemptImpl implements TaskAttempt,
.createSocketAddr(ta.nodeHttpAddress); // TODO: Costly?
ta.trackerName = StringInterner.weakIntern(nodeHttpInetAddr.getHostName());
ta.httpPort = nodeHttpInetAddr.getPort();
- ta.sendEvent(createJobCounterUpdateEventTALaunched(ta));
+ ta.sendEvent(createDAGCounterUpdateEventTALaunched(ta));
LOG.info("TaskAttempt: [" + ta.attemptId + "] started."
+ " Is using containerId: [" + ta.containerId + "]" + " on NM: ["
@@ -1316,7 +1300,8 @@ public class TaskAttemptImpl implements TaskAttempt,
ta.setFinishTime();
// Send out history event.
ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
- ta.sendEvent(createJobCounterUpdateEventSlotMillis(ta));
+ ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
+ TaskAttemptState.SUCCEEDED));
// Inform the Scheduler.
ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
@@ -1415,6 +1400,8 @@ public class TaskAttemptImpl implements TaskAttempt,
// TODO abort taskattempt
taskAttempt.sendEvent(new TaskEventTAUpdate(taskAttempt.attemptId,
TaskEventType.T_ATTEMPT_KILLED));
+ taskAttempt.sendEvent(createDAGCounterUpdateEventTAFinished(taskAttempt,
+ getExternalState(TaskAttemptStateInternal.KILLED)));
endState = TaskAttemptStateInternal.KILLED;
break;
case SUCCEEDED:
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
index c60ef58..9323270 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
@@ -25,7 +25,6 @@ import java.io.OutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.DagTypeConverters;
import org.apache.tez.dag.api.oldrecords.TaskState;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
@@ -45,7 +44,7 @@ public class TaskFinishedEvent implements HistoryEvent {
private TezCounters tezCounters;
private TezTaskAttemptID successfulAttemptID;
private String diagnostics;
-
+
public TaskFinishedEvent(TezTaskID taskID,
String vertexName, long startTime, long finishTime,
TezTaskAttemptID successfulAttemptID,
@@ -85,9 +84,6 @@ public class TaskFinishedEvent implements HistoryEvent {
if (diagnostics != null) {
builder.setDiagnostics(diagnostics);
}
- if (tezCounters != null) {
- builder.setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters));
- }
if (successfulAttemptID != null) {
builder.setSuccessfulTaskAttemptId(successfulAttemptID.toString());
}
@@ -101,10 +97,6 @@ public class TaskFinishedEvent implements HistoryEvent {
if (proto.hasDiagnostics()) {
this.diagnostics = proto.getDiagnostics();
}
- if (proto.hasCounters()) {
- this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
- proto.getCounters());
- }
if (proto.hasSuccessfulTaskAttemptId()) {
this.successfulAttemptID =
TezTaskAttemptID.fromString(proto.getSuccessfulTaskAttemptId());
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
index df3551a..8057714 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
@@ -25,7 +25,6 @@ import java.io.OutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.DagTypeConverters;
import org.apache.tez.dag.app.dag.VertexState;
import org.apache.tez.dag.app.dag.impl.VertexStats;
import org.apache.tez.dag.history.HistoryEvent;
@@ -99,9 +98,6 @@ public class VertexFinishedEvent implements HistoryEvent, SummaryEvent {
if (diagnostics != null) {
builder.setDiagnostics(diagnostics);
}
- if (tezCounters != null) {
- builder.setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters));
- }
return builder.build();
}
@@ -113,10 +109,6 @@ public class VertexFinishedEvent implements HistoryEvent, SummaryEvent {
if (proto.hasDiagnostics()) {
this.diagnostics = proto.getDiagnostics();
}
- if (proto.hasCounters()) {
- this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
- proto.getCounters());
- }
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-dag/src/main/proto/HistoryEvents.proto
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/proto/HistoryEvents.proto b/tez-dag/src/main/proto/HistoryEvents.proto
index 5cbe540..821612a 100644
--- a/tez-dag/src/main/proto/HistoryEvents.proto
+++ b/tez-dag/src/main/proto/HistoryEvents.proto
@@ -136,7 +136,6 @@ message VertexFinishedProto {
optional int64 finish_time = 3;
optional int32 state = 4;
optional string diagnostics = 5;
- optional TezCountersProto counters = 6;
}
message TaskStartedProto {
@@ -150,8 +149,7 @@ message TaskFinishedProto {
optional int64 finish_time = 2;
optional int32 state = 3;
optional string diagnostics = 4;
- optional TezCountersProto counters = 5;
- optional string successful_task_attempt_id = 6;
+ optional string successful_task_attempt_id = 5;
}
message TaskAttemptStartedProto {
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/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 3633a85..8e134f2 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
@@ -67,6 +67,7 @@ import org.apache.tez.dag.app.TaskHeartbeatHandler;
import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
import org.apache.tez.dag.app.dag.Vertex;
import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate;
import org.apache.tez.dag.app.dag.event.DAGEventType;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminated;
@@ -423,6 +424,9 @@ public class TestTaskAttempt {
verifyEventType(
arg.getAllValues().subList(expectedEventsAtRunning,
expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+ verifyEventType(
+ arg.getAllValues().subList(expectedEventsAtRunning,
+ expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
taImpl.handle(new TaskAttemptEventContainerTerminated(taskAttemptID,
"Terminated"));
@@ -571,6 +575,9 @@ public class TestTaskAttempt {
verifyEventType(
arg.getAllValues().subList(expectedEventsAtRunning,
expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+ verifyEventType(
+ arg.getAllValues().subList(expectedEventsAtRunning,
+ expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
taImpl.handle(new TaskAttemptEventContainerTerminated(taskAttemptID,
"Terminated"));
@@ -658,6 +665,9 @@ public class TestTaskAttempt {
verifyEventType(
arg.getAllValues().subList(expectedEventsAtRunning,
expectedEventsAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+ verifyEventType(
+ arg.getAllValues().subList(expectedEventsAtRunning,
+ expectedEventsAfterTerminating), DAGEventCounterUpdate.class, 1);
taImpl.handle(new TaskAttemptEvent(taskAttemptID,
TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM));
@@ -744,6 +754,9 @@ public class TestTaskAttempt {
verifyEventType(
arg.getAllValues().subList(expectedEventsAtRunning,
expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+ verifyEventType(
+ arg.getAllValues().subList(expectedEventsAtRunning,
+ expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
// Send out a Node Failure.
taImpl.handle(new TaskAttemptEventNodeFailed(taskAttemptID, "NodeDecomissioned"));
@@ -838,6 +851,9 @@ public class TestTaskAttempt {
verifyEventType(
arg.getAllValues().subList(expectedEventsAtRunning,
expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+ verifyEventType(
+ arg.getAllValues().subList(expectedEventsAtRunning,
+ expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
// Send out a Node Failure.
taImpl.handle(new TaskAttemptEventNodeFailed(taskAttemptID, "NodeDecomissioned"));
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
index 3b04cf6..a443a35 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
@@ -20,16 +20,17 @@ package org.apache.tez.dag.app.dag.impl;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
+import java.util.List;
+
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.tez.common.counters.TezCounters;
@@ -39,6 +40,7 @@ import org.apache.tez.dag.app.ContainerContext;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.TaskHeartbeatHandler;
import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate;
@@ -48,6 +50,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezTaskID;
import org.junit.Before;
import org.junit.Test;
+import org.mockito.ArgumentCaptor;
@SuppressWarnings({ "unchecked", "rawtypes" })
public class TestTaskAttemptRecovery {
@@ -98,6 +101,17 @@ public class TestTaskAttemptRecovery {
assertEquals(state, recoveredState);
}
+ private void verifyEvents(List<Event> events, Class<? extends Event> eventClass,
+ int expectedTimes) {
+ int actualTimes = 0;
+ for (Event event : events) {
+ if (eventClass.isInstance(event)) {
+ actualTimes ++;
+ }
+ }
+ assertEquals(expectedTimes, actualTimes);
+ }
+
/**
* No any event to restore -> RecoverTransition
*/
@@ -105,7 +119,14 @@ public class TestTaskAttemptRecovery {
public void testTARecovery_NEW() {
ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
- verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ assertEquals(2, events.size());
+ verifyEvents(events, TaskEventTAUpdate.class, 1);
+ // one for task killed
+ verifyEvents(events, DAGEventCounterUpdate.class, 1);
}
/**
@@ -117,7 +138,14 @@ public class TestTaskAttemptRecovery {
ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
- verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(3)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ assertEquals(3, events.size());
+ verifyEvents(events, TaskEventTAUpdate.class, 1);
+ // one for task launch, one for task killed
+ verifyEvents(events, DAGEventCounterUpdate.class, 2);
}
/**
@@ -131,7 +159,13 @@ public class TestTaskAttemptRecovery {
ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
assertEquals(TaskAttemptStateInternal.SUCCEEDED, ta.getInternalState());
- verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ assertEquals(2, events.size());
+ // one for task launch, one for task succeeded
+ verifyEvents(events, DAGEventCounterUpdate.class, 2);
}
/**
@@ -145,7 +179,13 @@ public class TestTaskAttemptRecovery {
ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
- verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ assertEquals(2, events.size());
+ // one for task launch, one for task killed
+ verifyEvents(events, DAGEventCounterUpdate.class, 2);
}
/**
@@ -159,7 +199,13 @@ public class TestTaskAttemptRecovery {
ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
assertEquals(TaskAttemptStateInternal.FAILED, ta.getInternalState());
- verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+
+ ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+ List<Event> events = eventCaptor.getAllValues();
+ assertEquals(2, events.size());
+ // one for task launch, one for task killed
+ verifyEvents(events, DAGEventCounterUpdate.class, 2);
}
/**
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
index 72d0c25..5bb7d35 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
@@ -354,7 +354,6 @@ public class TestHistoryEventsProtoConversion {
deserializedEvent.getFinishTime());
Assert.assertEquals(event.getState(), deserializedEvent.getState());
Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
- Assert.assertEquals(event.getTezCounters(), deserializedEvent.getTezCounters());
logEvents(event, deserializedEvent);
}
{
@@ -370,7 +369,6 @@ public class TestHistoryEventsProtoConversion {
deserializedEvent.getFinishTime());
Assert.assertEquals(event.getState(), deserializedEvent.getState());
Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
- Assert.assertEquals(event.getTezCounters(), deserializedEvent.getTezCounters());
logEvents(event, deserializedEvent);
}
}
@@ -403,8 +401,6 @@ public class TestHistoryEventsProtoConversion {
deserializedEvent.getFinishTime());
Assert.assertEquals(event.getState(),
deserializedEvent.getState());
- Assert.assertEquals(event.getTezCounters(),
- deserializedEvent.getTezCounters());
Assert.assertEquals(event.getSuccessfulAttemptID(),
deserializedEvent.getSuccessfulAttemptID());
Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
@@ -425,8 +421,6 @@ public class TestHistoryEventsProtoConversion {
deserializedEvent.getFinishTime());
Assert.assertEquals(event.getState(),
deserializedEvent.getState());
- Assert.assertEquals(event.getTezCounters(),
- deserializedEvent.getTezCounters());
Assert.assertEquals(event.getSuccessfulAttemptID(),
deserializedEvent.getSuccessfulAttemptID());
Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
index eae38f0..bd2fe99 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
@@ -24,6 +24,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
+import java.util.Map;
import java.util.Random;
import org.apache.commons.logging.Log;
@@ -38,6 +39,8 @@ import org.apache.tez.client.TezClient;
import org.apache.tez.client.TezClientUtils;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.DAGCounter;
+import org.apache.tez.common.counters.TezCounters;
import org.apache.tez.dag.api.DAG;
import org.apache.tez.dag.api.Edge;
import org.apache.tez.dag.api.EdgeProperty;
@@ -193,7 +196,12 @@ public class TestAMRecovery {
DAG dag =
createDAG(ControlledInputReadyVertexManager.class,
DataMovementType.BROADCAST, true);
- runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+ assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+ assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+ assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -220,7 +228,11 @@ public class TestAMRecovery {
DAG dag =
createDAG(ControlledInputReadyVertexManager.class,
DataMovementType.BROADCAST, false);
- runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+ assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+ assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+ assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -244,11 +256,15 @@ public class TestAMRecovery {
* @throws Exception
*/
@Test(timeout = 120000)
- public void testVertexPartialComplete_One2One() throws Exception {
+ public void testVertexPartialFinished_One2One() throws Exception {
DAG dag =
createDAG(ControlledInputReadyVertexManager.class,
DataMovementType.ONE_TO_ONE, true);
- runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+ assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+ assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+ assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -273,11 +289,15 @@ public class TestAMRecovery {
* @throws Exception
*/
@Test(timeout = 120000)
- public void testVertexCompletelyComplete_One2One() throws Exception {
+ public void testVertexCompletelyFinished_One2One() throws Exception {
DAG dag =
createDAG(ControlledInputReadyVertexManager.class,
DataMovementType.ONE_TO_ONE, false);
- runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+ assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+ assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+ assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -306,7 +326,11 @@ public class TestAMRecovery {
DAG dag =
createDAG(ControlledShuffleVertexManager.class,
DataMovementType.SCATTER_GATHER, true);
- runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+ assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+ assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+ assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -335,7 +359,11 @@ public class TestAMRecovery {
DAG dag =
createDAG(ControlledShuffleVertexManager.class,
DataMovementType.SCATTER_GATHER, false);
- runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+ assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+ assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+ assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+ assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -369,13 +397,14 @@ public class TestAMRecovery {
}
- void runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception {
+ TezCounters runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception {
tezSession.waitTillReady();
DAGClient dagClient = tezSession.submitDAG(dag);
DAGStatus dagStatus =
dagClient.waitForCompletionWithStatusUpdates(EnumSet
.of(StatusGetOpts.GET_COUNTERS));
Assert.assertEquals(finalState, dagStatus.getState());
+ return dagStatus.getDAGCounters();
}
/**
@@ -479,6 +508,19 @@ public class TestAMRecovery {
}
}
}
+
+ @Override
+ public void onVertexStarted(Map<String, List<Integer>> completions) {
+ // sleep for 1 seconds to delay the running of task in v2.
+ // this could keep the case that task of v1 is partial finished or completely
+ // finished, and at the same time the task of v2 is not started
+ try {
+ Thread.sleep(1*1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ super.onVertexStarted(completions);
+ }
}
public static class ControlledShuffleVertexManager extends
@@ -516,6 +558,19 @@ public class TestAMRecovery {
}
}
}
+
+ @Override
+ public void onVertexStarted(Map<String, List<Integer>> completions) {
+ // sleep for 1 seconds to delay the running of task in v2.
+ // this could keep the case that task of v1 is partial finished or completely
+ // finished, and at the same time the task of v2 is not started
+ try {
+ Thread.sleep(1*1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ super.onVertexStarted(completions);
+ }
}
public static class ControlledImmediateStartVertexManager extends
@@ -554,6 +609,19 @@ public class TestAMRecovery {
}
}
}
+
+ @Override
+ public void onVertexStarted(Map<String, List<Integer>> completions) {
+ // sleep for 1 seconds to delay the running of task in v2.
+ // this could keep the case that task of v1 is partial finished or completely
+ // finished, and at the same time the task of v2 is not started
+ try {
+ Thread.sleep(1*1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ super.onVertexStarted(completions);
+ }
}
/**
@@ -592,6 +660,10 @@ public class TestAMRecovery {
}
}
+ public static enum TestCounter {
+ Counter_1,
+ }
+
/**
* Do nothing if it is in task 0, sleep 3 seconds for other tasks. This enable
* us to kill AM in VM when some tasks are still running.
@@ -605,8 +677,10 @@ public class TestAMRecovery {
@Override
public void run() throws Exception {
+ getContext().getCounters().findCounter(TestCounter.Counter_1).increment(1);
if (getContext().getTaskIndex() == 0) {
- return;
+ // keep task_0 running for 1 seconds to wait for task_1 start running
+ Thread.sleep(1 * 1000);;
} else {
Thread.sleep(3 * 1000);
}
[08/25] TEZ-1539. Change InputInitializerEvent semantics to
SEND_ONCE_ON_TASK_SUCCESS. (sseth)
Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 04e2219..c003e05 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
@@ -75,6 +75,7 @@ import org.apache.tez.dag.api.OutputDescriptor;
import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.RootInputLeafOutput;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.VertexLocationHint;
import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
@@ -116,6 +117,7 @@ import org.apache.tez.dag.app.dag.event.DAGEventType;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
import org.apache.tez.dag.app.dag.event.TaskEvent;
+import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate;
import org.apache.tez.dag.app.dag.event.TaskEventType;
import org.apache.tez.dag.app.dag.event.VertexEvent;
import org.apache.tez.dag.app.dag.event.VertexEventRootInputFailed;
@@ -633,6 +635,208 @@ public class TestVertexImpl {
return dag;
}
+ private DAGPlan createDAGPlanWithRunningInitializer3() {
+ // v2 v1 (send event to v3)
+ // \ /
+ // \ /
+ // v3 -----(In)
+ // (Receive events from v1)
+ LOG.info("Setting up dag plan with running input initializer3");
+ DAGPlan dag = DAGPlan.newBuilder()
+ .setName("DagWithInputInitializer3")
+ .addVertex(
+ VertexPlan.newBuilder()
+ .setName("vertex1")
+ .setType(PlanVertexType.NORMAL)
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder()
+ .setNumTasks(1)
+ .setVirtualCores(1)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("x1.y1")
+ .build()
+ )
+ .addOutEdgeId("e1")
+ .build()
+ )
+ .addVertex(
+ VertexPlan.newBuilder()
+ .setName("vertex2")
+ .setType(PlanVertexType.NORMAL)
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder()
+ .setNumTasks(1)
+ .setVirtualCores(1)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("x1.y1")
+ .build()
+ )
+ .addOutEdgeId("e2")
+ .build()
+ )
+ .addVertex(
+ VertexPlan.newBuilder()
+ .setName("vertex3")
+ .setType(PlanVertexType.NORMAL)
+ .addInputs(
+ RootInputLeafOutputProto.newBuilder()
+ .setControllerDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName(
+ "IrrelevantInitializerClassName"))
+ .setName("input1")
+ .setIODescriptor(
+ TezEntityDescriptorProto.newBuilder()
+ .setClassName("InputClazz")
+ .build()
+ )
+ .build()
+ )
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder()
+ .setNumTasks(20)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("x2.y2")
+ .build()
+ )
+ .addInEdgeId("e1")
+ .addInEdgeId("e2")
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v1_v3"))
+ .setInputVertexName("vertex1")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+ .setOutputVertexName("vertex3")
+ .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+ .setId("e1")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v2_v3"))
+ .setInputVertexName("vertex2")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+ .setOutputVertexName("vertex3")
+ .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+ .setId("e2")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .build();
+ return dag;
+ }
+
+ private DAGPlan createDAGPlanWithRunningInitializer4() {
+ // v1 (send event to v3)
+ // |
+ // |
+ // v2 (In) (v2 can optioanlly send events to v2. Is setup via the initializer)
+ // | /
+ // | /
+ // v3 (Receive events from v1)
+ // Events are not generated by a directly connected vertex
+ LOG.info("Setting up dag plan with running input initializer4");
+ DAGPlan dag = DAGPlan.newBuilder()
+ .setName("DagWithInputInitializer4")
+ .addVertex(
+ VertexPlan.newBuilder()
+ .setName("vertex1")
+ .setType(PlanVertexType.NORMAL)
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder()
+ .setNumTasks(1)
+ .setVirtualCores(1)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("x1.y1")
+ .build()
+ )
+ .addOutEdgeId("e1")
+ .build()
+ )
+ .addVertex(
+ VertexPlan.newBuilder()
+ .setName("vertex2")
+ .setType(PlanVertexType.NORMAL)
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder()
+ .setNumTasks(2)
+ .setVirtualCores(1)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("x1.y1")
+ .build()
+ )
+ .addInEdgeId("e1")
+ .addOutEdgeId("e2")
+ .build()
+ )
+ .addVertex(
+ VertexPlan.newBuilder()
+ .setName("vertex3")
+ .setType(PlanVertexType.NORMAL)
+ .addInputs(
+ RootInputLeafOutputProto.newBuilder()
+ .setControllerDescriptor(
+ TezEntityDescriptorProto.newBuilder().setClassName(
+ "IrrelevantInitializerClassName"))
+ .setName("input1")
+ .setIODescriptor(
+ TezEntityDescriptorProto.newBuilder()
+ .setClassName("InputClazz")
+ .build()
+ )
+ .build()
+ )
+ .setTaskConfig(
+ PlanTaskConfiguration.newBuilder()
+ .setNumTasks(20)
+ .setVirtualCores(4)
+ .setMemoryMb(1024)
+ .setJavaOpts("")
+ .setTaskModule("x2.y2")
+ .build()
+ )
+ .addInEdgeId("e2")
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v1_v3"))
+ .setInputVertexName("vertex1")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+ .setOutputVertexName("vertex2")
+ .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+ .setId("e1")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .addEdge(
+ EdgePlan.newBuilder()
+ .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v2_v3"))
+ .setInputVertexName("vertex2")
+ .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+ .setOutputVertexName("vertex3")
+ .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+ .setId("e2")
+ .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+ .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+ .build()
+ )
+ .build();
+ return dag;
+ }
+
+
private DAGPlan createDAGPlanWithRunningInitializer() {
LOG.info("Setting up dag plan with running input initializer");
DAGPlan dag = DAGPlan.newBuilder()
@@ -643,7 +847,7 @@ public class TestVertexImpl {
.setType(PlanVertexType.NORMAL)
.setTaskConfig(
PlanTaskConfiguration.newBuilder()
- .setNumTasks(10)
+ .setNumTasks(1)
.setVirtualCores(4)
.setMemoryMb(1024)
.setJavaOpts("")
@@ -1742,7 +1946,7 @@ public class TestVertexImpl {
for (PlanVertexGroupInfo groupInfo : dagPlan.getVertexGroupsList()) {
vertexGroups.put(groupInfo.getGroupName(), new VertexGroupInfo(groupInfo));
}
- updateTracker = new StateChangeNotifier(dag);
+ updateTracker = new StateChangeNotifier(appContext.getCurrentDAG());
setupVertices();
when(dag.getVertex(any(TezVertexID.class))).thenAnswer(new Answer<Vertex>() {
@Override
@@ -2962,16 +3166,390 @@ public class TestVertexImpl {
Assert.assertEquals(VertexState.SUCCEEDED, v1.getState());
// At this point, 2 events should have been received - since the dispatcher is complete.
- Assert.assertEquals(2, initializer.stateUpdateEvents.size());
+ Assert.assertEquals(2, initializer.stateUpdates.size());
+ Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.RUNNING,
+ initializer.stateUpdates.get(0).getVertexState());
+ Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.SUCCEEDED,
+ initializer.stateUpdates.get(1).getVertexState());
+ }
+
+ @Test(timeout = 1000000)
+ public void testInputInitializerEventMultipleAttempts() throws Exception {
+ useCustomInitializer = true;
+ customInitializer = new EventHandlingRootInputInitializer(null);
+ EventHandlingRootInputInitializer initializer =
+ (EventHandlingRootInputInitializer) customInitializer;
+ setupPreDagCreation();
+ dagPlan = createDAGPlanWithRunningInitializer4();
+ setupPostDagCreation();
+
+ VertexImplWithRunningInputInitializer v1 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+ VertexImplWithRunningInputInitializer v2 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+ VertexImplWithRunningInputInitializer v3 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+ initVertex(v1);
+ startVertex(v1);
+ dispatcher.await();
+
+ // Vertex1 start should trigger downstream vertices
+ Assert.assertEquals(VertexState.RUNNING, v1.getState());
+ Assert.assertEquals(VertexState.RUNNING, v2.getState());
+ Assert.assertEquals(VertexState.INITIALIZING, v3.getState());
+
+ ByteBuffer expected;
+
+ // Genrate events from v1 to v3's InputInitializer
+ ByteBuffer payload = ByteBuffer.allocate(12).putInt(0, 1).putInt(4, 0).putInt(8, 0);
+ InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", payload);
+ // Create taskId and taskAttemptId for the single task that exists in vertex1
+ TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+ TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+ TezEvent tezEvent = new TezEvent(event,
+ new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+ dispatcher.getEventHandler()
+ .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+ dispatcher.await();
+
+ payload = ByteBuffer.allocate(12).putInt(0, 1).putInt(4, 0).putInt(8, 1);
+ expected = payload;
+ event = InputInitializerEvent.create("vertex3", "input1", payload);
+ // Create taskId and taskAttemptId for the single task that exists in vertex1
+ TezTaskAttemptID ta1_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 1);
+ tezEvent = new TezEvent(event,
+ new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta1_t0_v1));
+ dispatcher.getEventHandler()
+ .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+ dispatcher.await();
+
+ // Events should not be cached in the vertex, since the initializer is running
+ Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+ // Events should be cached since the tasks have not succeeded.
+ // Verify that events are cached
+ RootInputInitializerManager.InitializerWrapper initializerWrapper =
+ v3.rootInputInitializerManager.getInitializerWrapper("input1");
+ Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+ Assert.assertEquals(2, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+
+ // Get all tasks of vertex1 to succeed.
+ for (TezTaskID taskId : v1.getTasks().keySet()) {
+ // Make attempt 1 of every task succeed
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 1);
+ v1.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+ v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+ dispatcher.await();
+ v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+ }
+ dispatcher.await();
+
+ // v3 would have processed an INIT event and moved into INITIALIZING state.
+ // Since source tasks were complete - the events should have been consumed.
+ // Initializer would have run, and processed events.
+ while (v3.getState() != VertexState.RUNNING) {
+ Thread.sleep(10);
+ }
+ Assert.assertEquals(VertexState.RUNNING, v3.getState());
+
+ Assert.assertEquals(1, initializer.initializerEvents.size());
+ Assert.assertEquals(expected, initializer.initializerEvents.get(0).getUserPayload());
+
+ }
+
+ @Test(timeout = 10000)
+ public void testInputInitializerEventsMultipleSources() throws Exception {
+ useCustomInitializer = true;
+ customInitializer = new EventHandlingRootInputInitializer(null);
+ EventHandlingRootInputInitializer initializer =
+ (EventHandlingRootInputInitializer) customInitializer;
+ initializer.setNumExpectedEvents(4);
+ setupPreDagCreation();
+ dagPlan = createDAGPlanWithRunningInitializer4();
+ setupPostDagCreation();
+
+ VertexImplWithRunningInputInitializer v1 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+ VertexImplWithRunningInputInitializer v2 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+ VertexImplWithRunningInputInitializer v3 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+ initVertex(v1);
+ startVertex(v1);
+ dispatcher.await();
+
+ // Vertex1 start should trigger downstream vertices
+ Assert.assertEquals(VertexState.RUNNING, v1.getState());
+ Assert.assertEquals(VertexState.RUNNING, v2.getState());
+ Assert.assertEquals(VertexState.INITIALIZING, v3.getState());
+
+ List<ByteBuffer> expectedPayloads = new LinkedList<ByteBuffer>();
+
+ // Genrate events from v1 to v3's InputInitializer
+ ByteBuffer payload = ByteBuffer.allocate(12).putInt(0, 1).putInt(4, 0).putInt(8, 0);
+ expectedPayloads.add(payload);
+ InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", payload);
+ // Create taskId and taskAttemptId for the single task that exists in vertex1
+ TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+ TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+ TezEvent tezEvent = new TezEvent(event,
+ new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+ dispatcher.getEventHandler()
+ .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+ dispatcher.await();
+
+ // Events should not be cached in the vertex, since the initializer is running
+ Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+ // Events should be cached since the tasks have not succeeded.
+ // Verify that events are cached
+ RootInputInitializerManager.InitializerWrapper initializerWrapper =
+ v3.rootInputInitializerManager.getInitializerWrapper("input1");
+ Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+ Assert.assertEquals(1, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+
+ // Get all tasks of vertex1 to succeed.
+ for (TezTaskID taskId : v1.getTasks().keySet()) {
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+ v1.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+ v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+ dispatcher.await();
+ v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+ }
+ dispatcher.await();
+
+ Assert.assertEquals(1, initializer.initializerEvents.size());
+
+
+ // Test written based on this
+ Assert.assertEquals(2, v2.getTotalTasks());
+ // Generate events from v2 to v3's initializer. 1 from task 0, 2 from task 1
+ for (Task task : v2.getTasks().values()) {
+ TezTaskID taskId = task.getTaskId();
+ TezTaskAttemptID attemptId = TezTaskAttemptID.getInstance(taskId, 0);
+ int numEventsFromTask = taskId.getId() + 1;
+ for (int i = 0; i < numEventsFromTask; i++) {
+ payload = ByteBuffer.allocate(12).putInt(0, 2).putInt(4, taskId.getId()).putInt(8, i);
+ expectedPayloads.add(payload);
+ InputInitializerEvent event2 = InputInitializerEvent.create("vertex3", "input1", payload);
+ TezEvent tezEvent2 = new TezEvent(event2,
+ new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex2", "vertex3", attemptId));
+ dispatcher.getEventHandler()
+ .handle(
+ new VertexEventRouteEvent(v2.getVertexId(), Collections.singletonList(tezEvent2)));
+ dispatcher.await();
+ }
+ }
+
+ // Validate queueing of these events
+ // Only v2 events pending
+ Assert.assertEquals(1, initializerWrapper.getPendingEvents().keySet().size());
+ // 3 events pending
+ Assert.assertEquals(3, initializerWrapper.getPendingEvents().get(v2.getName()).size());
+
+ // Get all tasks of vertex1 to succeed.
+ for (TezTaskID taskId : v2.getTasks().keySet()) {
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+ v2.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+ v2.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+ dispatcher.await();
+ v2.stateChangeNotifier.taskSucceeded(v2.getName(), taskId, taskAttemptId.getId());
+ }
+ dispatcher.await();
+
+ // v3 would have processed an INIT event and moved into INITIALIZING state.
+ // Since source tasks were complete - the events should have been consumed.
+ // Initializer would have run, and processed events.
+ while (v3.getState() != VertexState.RUNNING) {
+ Thread.sleep(10);
+ }
+ Assert.assertEquals(VertexState.RUNNING, v3.getState());
+
+ Assert.assertEquals(4, initializer.initializerEvents.size());
+ Assert.assertTrue(initializer.initComplete.get());
+
+ Assert.assertEquals(2, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+ Assert.assertEquals(0, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+ for (InputInitializerEvent initializerEvent : initializer.initializerEvents) {
+ expectedPayloads.remove(initializerEvent.getUserPayload());
+ }
+ Assert.assertEquals(0, expectedPayloads.size());
+
+ }
+
+ @Test(timeout = 10000)
+ public void testInputInitializerEventNoDirectConnection() throws Exception {
+ useCustomInitializer = true;
+ customInitializer = new EventHandlingRootInputInitializer(null);
+ EventHandlingRootInputInitializer initializer =
+ (EventHandlingRootInputInitializer) customInitializer;
+ setupPreDagCreation();
+ dagPlan = createDAGPlanWithRunningInitializer4();
+ setupPostDagCreation();
+
+ VertexImplWithRunningInputInitializer v1 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+ VertexImplWithRunningInputInitializer v2 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+ VertexImplWithRunningInputInitializer v3 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+ initVertex(v1);
+ startVertex(v1);
+ dispatcher.await();
+
+ // Vertex1 start should trigger downstream vertices
+ Assert.assertEquals(VertexState.RUNNING, v1.getState());
+ Assert.assertEquals(VertexState.RUNNING, v2.getState());
+ Assert.assertEquals(VertexState.INITIALIZING, v3.getState());
+
+ // Genrate events from v1 to v3's InputInitializer
+ InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", null);
+ // Create taskId and taskAttemptId for the single task that exists in vertex1
+ TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+ TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+ TezEvent tezEvent = new TezEvent(event,
+ new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+ dispatcher.getEventHandler()
+ .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+ dispatcher.await();
+
+ // Events should not be cached in the vertex, since the initializer is running
+ Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+ // Events should be cached since the tasks have not succeeded.
+ // Verify that events are cached
+ RootInputInitializerManager.InitializerWrapper initializerWrapper =
+ v3.rootInputInitializerManager.getInitializerWrapper("input1");
+ Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+ Assert.assertEquals(1, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+ // Get all tasks of vertex1 to succeed.
+ for (TezTaskID taskId : v1.getTasks().keySet()) {
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+ v1.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+ v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+ dispatcher.await();
+ v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+ }
+ dispatcher.await();
+
+ // v3 would have processed an INIT event and moved into INITIALIZING state.
+ // Since source tasks were complete - the events should have been consumed.
+ // Initializer would have run, and processed events.
+ while (v3.getState() != VertexState.RUNNING) {
+ Thread.sleep(10);
+ }
+
+ Assert.assertEquals(VertexState.RUNNING, v3.getState());
+
+ Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+ Assert.assertEquals(0, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+ Assert.assertTrue(initializer.eventReceived.get());
+ Assert.assertEquals(2, initializer.stateUpdates.size());
Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.RUNNING,
- initializer.stateUpdateEvents.get(0).getVertexState());
+ initializer.stateUpdates.get(0).getVertexState());
Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.SUCCEEDED,
- initializer.stateUpdateEvents.get(1).getVertexState());
+ initializer.stateUpdates.get(1).getVertexState());
+ }
+
+ @Test(timeout = 10000)
+ public void testInputInitializerEventsAtNew() throws Exception {
+ useCustomInitializer = true;
+ customInitializer = new EventHandlingRootInputInitializer(null);
+ EventHandlingRootInputInitializer initializer =
+ (EventHandlingRootInputInitializer) customInitializer;
+ setupPreDagCreation();
+ dagPlan = createDAGPlanWithRunningInitializer3();
+ setupPostDagCreation();
+
+ VertexImplWithRunningInputInitializer v1 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+ VertexImplWithRunningInputInitializer v2 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+ VertexImplWithRunningInputInitializer v3 =
+ (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+ initVertex(v1);
+ startVertex(v1);
+ dispatcher.await();
+
+ // Vertex2 has not been INITED, so the rest of the vertices should be in state NEW.
+ Assert.assertEquals(VertexState.RUNNING, v1.getState());
+ Assert.assertEquals(VertexState.NEW, v2.getState());
+ Assert.assertEquals(VertexState.NEW, v3.getState());
+
+ // Genrate events from v1 to v3's InputInitializer
+ InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", null);
+ // Create taskId and taskAttemptId for the single task that exists in vertex1
+ TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+ TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+ TezEvent tezEvent = new TezEvent(event,
+ new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+ dispatcher.getEventHandler()
+ .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+ dispatcher.await();
+
+ // Events should be cached in the vertex, since the Initializer has not started
+ Assert.assertEquals(1, v3.pendingInitializerEvents.size());
+
+ // Get Vertex1 to succeed before Vertex2 is INITED. Contrived case ? This is likely a tiny race.
+ for (TezTaskID taskId : v1.getTasks().keySet()) {
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+ TaskImpl task = (TaskImpl)v1.getTask(taskId);
+ task.handle(new TaskEvent(taskId, TaskEventType.T_ATTEMPT_LAUNCHED));
+ task.handle(new TaskEventTAUpdate(taskAttemptId, TaskEventType.T_ATTEMPT_SUCCEEDED));
+ v1.handle(new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+ v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+ dispatcher.await();
+ v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+ }
+ dispatcher.await();
+
+ // Events should still be cached in the vertex
+ Assert.assertEquals(1, v3.pendingInitializerEvents.size());
+ Assert.assertEquals(VertexState.NEW, v3.getState());
+
+ // Move processing along. INIT the remaining root level vertex.
+ initVertex(v2);
+ startVertex(v2);
+ dispatcher.await();
+
+
+ // v3 would have processed an INIT event and moved into INITIALIZING state.
+ // Since source tasks were complete - the events should have been consumed.
+ // Initializer would have run, and processed events.
+ while (v3.getState() != VertexState.RUNNING) {
+ Thread.sleep(10);
+ }
+
+ Assert.assertEquals(VertexState.RUNNING, v3.getState());
+ // Events should have been cleared from the vertex.
+ Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+ // KK Add checks to validate thte RootInputManager doesn't remember the events either
+
+ Assert.assertTrue(initializer.eventReceived.get());
+ Assert.assertEquals(2, initializer.stateUpdates.size());
+ Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.RUNNING,
+ initializer.stateUpdates.get(0).getVertexState());
+ Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.SUCCEEDED,
+ initializer.stateUpdates.get(1).getVertexState());
}
@SuppressWarnings("unchecked")
@Test(timeout = 10000)
- public void testRootInputInitializerEvent() throws Exception {
+ public void testInputInitializerEvents() throws Exception {
useCustomInitializer = true;
customInitializer = new EventHandlingRootInputInitializer(null);
EventHandlingRootInputInitializer initializer =
@@ -2998,15 +3576,36 @@ public class TestVertexImpl {
Assert.assertFalse(initializer.eventReceived.get());
Assert.assertFalse(initializer.initComplete.get());
+
// Signal the initializer by sending an event - via vertex1
InputInitializerEvent event = InputInitializerEvent.create("vertex2", "input1", null);
+ // Create taskId and taskAttemptId for the single task that exists in vertex1
+ TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+ TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
TezEvent tezEvent = new TezEvent(event,
- new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", null, null));
+ new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex2", ta0_t0_v1));
dispatcher.getEventHandler()
.handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
dispatcher.await();
+ // Events should not be cached in the vertex, since the initializer is running
+ Assert.assertEquals(0, v2.pendingInitializerEvents.size());
+
+ // Verify that events are cached
+ RootInputInitializerManager.InitializerWrapper initializerWrapper =
+ v2.rootInputInitializerManager.getInitializerWrapper("input1");
+ Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+ Assert.assertEquals(1, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+ for (TezTaskID taskId : v1.getTasks().keySet()) {
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+ v1.handle(new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+ v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+ dispatcher.await();
+ v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+ }
+
// Both happening in separate threads
while (!initializer.eventReceived.get()) {
Thread.sleep(10);
@@ -3019,6 +3618,10 @@ public class TestVertexImpl {
while (v2.getState() != VertexState.RUNNING) {
Thread.sleep(10);
}
+
+ // Verify the events are no longer cached, but attempts are remembered
+ Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+ Assert.assertEquals(0, initializerWrapper.getPendingEvents().get(v1.getName()).size());
}
@Test(timeout = 5000)
@@ -3358,7 +3961,7 @@ public class TestVertexImpl {
VertexImplWithControlledInitializerManager v2 = (VertexImplWithControlledInitializerManager) vertices.get("vertex2");
Assert.assertEquals(VertexState.INITIALIZING, v2.getState());
- // non-task events dont get buffered
+ // non-task events don't get buffered
List<TezEvent> events = Lists.newLinkedList();
TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
@@ -3954,8 +4557,10 @@ public class TestVertexImpl {
private final ReentrantLock lock = new ReentrantLock();
private final Condition eventCondition = lock.newCondition();
- private final List<VertexStateUpdate> stateUpdateEvents = new LinkedList<VertexStateUpdate>();
+ private final List<VertexStateUpdate> stateUpdates = new LinkedList<VertexStateUpdate>();
+ private final List<InputInitializerEvent> initializerEvents = new LinkedList<InputInitializerEvent>();
private volatile InputInitializerContext context;
+ private volatile int numExpectedEvents = 1;
public EventHandlingRootInputInitializer(
InputInitializerContext initializerContext) {
@@ -3968,7 +4573,9 @@ public class TestVertexImpl {
initStarted.set(true);
lock.lock();
try {
- eventCondition.await();
+ if (!eventReceived.get()) {
+ eventCondition.await();
+ }
} finally {
lock.unlock();
}
@@ -3983,12 +4590,15 @@ public class TestVertexImpl {
@Override
public void handleInputInitializerEvent(List<InputInitializerEvent> events) throws
Exception {
- eventReceived.set(true);
- lock.lock();
- try {
- eventCondition.signal();
- } finally {
- lock.unlock();
+ initializerEvents.addAll(events);
+ if (initializerEvents.size() == numExpectedEvents) {
+ eventReceived.set(true);
+ lock.lock();
+ try {
+ eventCondition.signal();
+ } finally {
+ lock.unlock();
+ }
}
}
@@ -3997,8 +4607,12 @@ public class TestVertexImpl {
this.context = context;
}
+ public void setNumExpectedEvents(int numEvents) {
+ this.numExpectedEvents = numEvents;
+ }
+
public void onVertexStateUpdated(VertexStateUpdate stateUpdate) {
- stateUpdateEvents.add(stateUpdate);
+ stateUpdates.add(stateUpdate);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
index e2f189c..9042a93 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
@@ -62,7 +62,7 @@ import org.apache.tez.dag.app.dag.event.VertexEvent;
import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
import org.apache.tez.dag.app.dag.event.VertexEventType;
import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter;
-import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
import org.apache.tez.dag.history.events.VertexFinishedEvent;
import org.apache.tez.dag.history.events.VertexInitializedEvent;
import org.apache.tez.dag.history.events.VertexStartedEvent;
@@ -403,7 +403,7 @@ public class TestVertexRecovery {
public void testRecovery_New_Desired_RUNNING() {
VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
VertexState recoveredState =
- vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
assertEquals(VertexState.NEW, recoveredState);
assertEquals(1, vertex1.recoveredEvents.size());
@@ -478,7 +478,7 @@ public class TestVertexRecovery {
restoreFromInitializedEvent(vertex1);
VertexState recoveredState =
- vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
assertEquals(VertexState.INITED, recoveredState);
@@ -522,7 +522,7 @@ public class TestVertexRecovery {
assertEquals(startedTime, vertex1.startedTime);
recoveredState =
- vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
assertEquals(VertexState.RUNNING, recoveredState);
assertEquals(1, vertex1.recoveredEvents.size());
@@ -616,7 +616,7 @@ public class TestVertexRecovery {
VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
VertexState recoveredState =
- vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
assertEquals(VertexState.NEW, recoveredState);
assertEquals(1, vertex3.recoveredEvents.size());
@@ -669,7 +669,7 @@ public class TestVertexRecovery {
VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
VertexState recoveredState =
- vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
assertEquals(VertexState.NEW, recoveredState);
assertEquals(1, vertex3.recoveredEvents.size());
@@ -734,7 +734,7 @@ public class TestVertexRecovery {
VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
recoveredState =
- vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
assertEquals(VertexState.NEW, recoveredState);
assertEquals(1, vertex3.recoveredEvents.size());
@@ -814,7 +814,7 @@ public class TestVertexRecovery {
VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
recoveredState =
- vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+ vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
assertEquals(VertexState.NEW, recoveredState);
assertEquals(1, vertex3.recoveredEvents.size());
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
index 5bb7d35..bcbe6f1 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
@@ -527,9 +527,9 @@ public class TestHistoryEventsProtoConversion {
}
private void testVertexDataMovementEventsGeneratedEvent() throws Exception {
- VertexDataMovementEventsGeneratedEvent event;
+ VertexRecoverableEventsGeneratedEvent event;
try {
- event = new VertexDataMovementEventsGeneratedEvent(
+ event = new VertexRecoverableEventsGeneratedEvent(
TezVertexID.getInstance(
TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 1), null);
Assert.fail("Invalid creation should have errored out");
@@ -539,11 +539,11 @@ public class TestHistoryEventsProtoConversion {
List<TezEvent> events =
Arrays.asList(new TezEvent(DataMovementEvent.create(1, null),
new EventMetaData(EventProducerConsumerType.SYSTEM, "foo", "bar", null)));
- event = new VertexDataMovementEventsGeneratedEvent(
+ event = new VertexRecoverableEventsGeneratedEvent(
TezVertexID.getInstance(
TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 1), events);
- VertexDataMovementEventsGeneratedEvent deserializedEvent =
- (VertexDataMovementEventsGeneratedEvent) testProtoConversion(event);
+ VertexRecoverableEventsGeneratedEvent deserializedEvent =
+ (VertexRecoverableEventsGeneratedEvent) testProtoConversion(event);
Assert.assertEquals(event.getVertexID(), deserializedEvent.getVertexID());
Assert.assertEquals(1,
deserializedEvent.getTezEvents().size());
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
index ad39531..f674fc0 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
@@ -45,7 +45,7 @@ import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
import org.apache.tez.dag.history.events.TaskFinishedEvent;
import org.apache.tez.dag.history.events.TaskStartedEvent;
import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
import org.apache.tez.dag.history.events.VertexFinishedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
@@ -152,7 +152,7 @@ public class TestHistoryEventJsonConversion {
event = new ContainerStoppedEvent(containerId, random.nextInt(), -1, applicationAttemptId);
break;
case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
- event = new VertexDataMovementEventsGeneratedEvent();
+ event = new VertexRecoverableEventsGeneratedEvent();
break;
case DAG_COMMIT_STARTED:
event = new DAGCommitStartedEvent();
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
index 5b19e80..b04b8d4 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
@@ -45,7 +45,7 @@ import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
import org.apache.tez.dag.history.events.TaskFinishedEvent;
import org.apache.tez.dag.history.events.TaskStartedEvent;
import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
import org.apache.tez.dag.history.events.VertexFinishedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
@@ -152,7 +152,7 @@ public class TestHistoryEventTimelineConversion {
event = new ContainerStoppedEvent(containerId, random.nextInt(), -1, applicationAttemptId);
break;
case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
- event = new VertexDataMovementEventsGeneratedEvent();
+ event = new VertexRecoverableEventsGeneratedEvent();
break;
case DAG_COMMIT_STARTED:
event = new DAGCommitStartedEvent();
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
index 7676313..9595cb9 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
@@ -42,8 +42,8 @@ import org.apache.tez.dag.api.client.DAGStatus.State;
import org.apache.tez.dag.app.RecoveryParser;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
-import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
import org.apache.tez.test.dag.MultiAttemptDAG;
import org.apache.tez.test.dag.MultiAttemptDAG.FailingInputInitializer;
import org.apache.tez.test.dag.MultiAttemptDAG.NoOpInput;
@@ -198,8 +198,8 @@ public class TestDAGRecovery {
+ ", eventType=" + historyEvent.getEventType()
+ ", event=" + historyEvent);
if (historyEvent.getEventType() == HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED) {
- VertexDataMovementEventsGeneratedEvent dmEvent =
- (VertexDataMovementEventsGeneratedEvent)historyEvent;
+ VertexRecoverableEventsGeneratedEvent dmEvent =
+ (VertexRecoverableEventsGeneratedEvent) historyEvent;
// TODO do not need to check whether it is -1 after Tez-1521 is resolved
if (dmEvent.getVertexID().getId() == 0 && inputInfoEventIndex == -1) {
inputInfoEventIndex = j;
[20/25] git commit: TEZ-1571. Add create method for
DataSinkDescriptor. Contributed by Jeff Zhang.
Posted by je...@apache.org.
TEZ-1571. Add create method for DataSinkDescriptor. Contributed by Jeff
Zhang.
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/dbe6a38a
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/dbe6a38a
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/dbe6a38a
Branch: refs/heads/TEZ-8
Commit: dbe6a38a5646841fa60e103566d3a54cc8f3065e
Parents: 5d6b8fd
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 15 22:36:15 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 15 22:36:15 2014 -0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/tez/dag/api/DataSinkDescriptor.java | 24 ++++++++++++++++++++
.../org/apache/tez/dag/api/TestDAGVerify.java | 10 ++++----
.../tez/dag/app/dag/impl/TestDAGImpl.java | 4 ++--
.../tez/dag/history/utils/TestDAGUtils.java | 4 ++--
.../apache/tez/mapreduce/client/YARNRunner.java | 2 +-
.../apache/tez/mapreduce/output/MROutput.java | 2 +-
.../mapreduce/examples/FilterLinesByWord.java | 2 +-
.../examples/FilterLinesByWordOneToOne.java | 2 +-
.../org/apache/tez/test/TestDAGRecovery2.java | 2 +-
10 files changed, 39 insertions(+), 14 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1520e70..97cf79b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -40,6 +40,7 @@ ALL CHANGES
TEZ-1574. Support additional formats for the tez deployed archive
TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
of DAG submission
+ TEZ-1571. Add create method for DataSinkDescriptor.
Release 0.5.0: 2014-09-03
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
index 2240384..bc43c88 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
@@ -51,6 +51,7 @@ public class DataSinkDescriptor {
* operation.
* @param credentials Credentials needs to access the data sink
*/
+ @Deprecated
public DataSinkDescriptor(OutputDescriptor outputDescriptor,
@Nullable OutputCommitterDescriptor committerDescriptor,
@Nullable Credentials credentials) {
@@ -59,6 +60,29 @@ public class DataSinkDescriptor {
this.credentials = credentials;
}
+ /**
+ * Create a {@link DataSinkDescriptor}
+ * @param outputDescriptor
+ * An {@link OutputDescriptor} for the output
+ * @param committerDescriptor
+ * Specify a committer to be used for the output. Can be null. After all
+ * tasks in the vertex (or in the DAG) have completed, the committer
+ * (if specified) is invoked to commit the outputs. Commit is a data
+ * sink specific operation that usually determines the visibility of
+ * the output to external observers. E.g. moving output files from
+ * temporary dirs to the real output dir. When there are multiple
+ * executions of a task, the commit process also helps decide which
+ * execution will be included in the final output. Users should
+ * consider whether their application or data sink need a commit
+ * operation.
+ * @param credentials Credentials needs to access the data sink
+ */
+ public static DataSinkDescriptor create(OutputDescriptor outputDescriptor,
+ @Nullable OutputCommitterDescriptor committerDescriptor,
+ @Nullable Credentials credentials) {
+ return new DataSinkDescriptor(outputDescriptor, committerDescriptor, credentials);
+ }
+
public OutputDescriptor getOutputDescriptor() {
return outputDescriptor;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
index bf9abf1..b0b6d29 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
@@ -521,7 +521,7 @@ public class TestDAGVerify {
ProcessorDescriptor.create("MapProcessor"),
dummyTaskCount, dummyTaskResource);
- v1.addDataSink("v2", new DataSinkDescriptor(null, null, null));
+ v1.addDataSink("v2", DataSinkDescriptor.create(null, null, null));
Edge e1 = Edge.create(v1, v2,
EdgeProperty.create(DataMovementType.SCATTER_GATHER,
@@ -545,7 +545,7 @@ public class TestDAGVerify {
ProcessorDescriptor.create("MapProcessor"),
dummyTaskCount, dummyTaskResource);
- v1.addDataSink("v2", new DataSinkDescriptor(null, null, null));
+ v1.addDataSink("v2", DataSinkDescriptor.create(null, null, null));
DAG dag = DAG.create("testDag");
dag.addVertex(v1);
@@ -621,7 +621,7 @@ public class TestDAGVerify {
DAG dag = DAG.create("testDag");
VertexGroup uv12 = dag.createVertexGroup("uv12", v1, v2);
OutputDescriptor outDesc = new OutputDescriptor();
- uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, null, null));
+ uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null));
GroupInputEdge e1 = GroupInputEdge.create(uv12, v3,
EdgeProperty.create(DataMovementType.SCATTER_GATHER,
@@ -678,7 +678,7 @@ public class TestDAGVerify {
String groupName1 = "uv12";
VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2);
OutputDescriptor outDesc = new OutputDescriptor();
- uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, null, null));
+ uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null));
String groupName2 = "uv23";
VertexGroup uv23 = dag.createVertexGroup(groupName2, v2, v3);
@@ -760,7 +760,7 @@ public class TestDAGVerify {
String groupName1 = "uv12";
VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2);
OutputDescriptor outDesc = new OutputDescriptor();
- uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, null, null));
+ uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null));
String groupName2 = "uv23";
VertexGroup uv23 = dag.createVertexGroup(groupName2, v2, v3);
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/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 aba4fd9..03aedef 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
@@ -364,8 +364,8 @@ public class TestDAGImpl {
TotalCountingOutputCommitter.class.getName());
org.apache.tez.dag.api.VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2);
OutputDescriptor outDesc = OutputDescriptor.create("output.class");
- uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
- v3.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
+ uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
+ v3.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
GroupInputEdge e1 = GroupInputEdge.create(uv12, v3,
EdgeProperty.create(DataMovementType.SCATTER_GATHER,
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java b/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
index 4a0120b..081ed10 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
@@ -73,8 +73,8 @@ public class TestDAGUtils {
.setHistoryText("uvOut HistoryText");
OutputCommitterDescriptor ocd =
OutputCommitterDescriptor.create(OutputCommitter.class.getName());
- uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
- v3.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
+ uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
+ v3.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
GroupInputEdge e1 = GroupInputEdge.create(uv12, v3,
EdgeProperty.create(DataMovementType.SCATTER_GATHER,
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
index d6c8057..dfbf0cf 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
@@ -428,7 +428,7 @@ public class YARNRunner implements ClientProtocol {
if (stageNum == totalStages -1) {
OutputDescriptor od = OutputDescriptor.create(MROutputLegacy.class.getName())
.setUserPayload(vertexUserPayload);
- vertex.addDataSink("MROutput", new DataSinkDescriptor(od,
+ vertex.addDataSink("MROutput", DataSinkDescriptor.create(od,
OutputCommitterDescriptor.create(MROutputCommitter.class.getName()), null));
}
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/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 db9fbea..ab9b41d 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
@@ -179,7 +179,7 @@ public class MROutput extends AbstractLogicalOutput {
}
}
- return new DataSinkDescriptor(
+ return DataSinkDescriptor.create(
OutputDescriptor.create(outputClassName).setUserPayload(createUserPayload()),
(doCommit ? OutputCommitterDescriptor.create(
MROutputCommitter.class.getName()) : null), credentials);
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
index 9f533d9..8430c68 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
@@ -196,7 +196,7 @@ public class FilterLinesByWord extends Configured implements Tool {
.setUserPayload(TezUtils.createUserPayloadFromConf(stage2Conf));
OutputCommitterDescriptor ocd =
OutputCommitterDescriptor.create(MROutputCommitter.class.getName());
- stage2Vertex.addDataSink("MROutput", new DataSinkDescriptor(od, ocd, null));
+ stage2Vertex.addDataSink("MROutput", DataSinkDescriptor.create(od, ocd, null));
UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
.newBuilder(Text.class.getName(), TextLongPair.class.getName()).build();
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
index e038170..515cf70 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
@@ -185,7 +185,7 @@ public class FilterLinesByWordOneToOne extends Configured implements Tool {
// Configure the Output for stage2
stage2Vertex.addDataSink(
"MROutput",
- new DataSinkDescriptor(OutputDescriptor.create(MROutput.class.getName())
+ DataSinkDescriptor.create(OutputDescriptor.create(MROutput.class.getName())
.setUserPayload(TezUtils.createUserPayloadFromConf(stage2Conf)),
OutputCommitterDescriptor.create(MROutputCommitter.class.getName()), null));
http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
index 37bea80..b7957a3 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
@@ -183,7 +183,7 @@ public class TestDAGRecovery2 {
.toUserPayload())));
OutputCommitterDescriptor ocd = OutputCommitterDescriptor.create(
MultiAttemptDAG.FailingOutputCommitter.class.getName());
- dag.getVertex("v3").addDataSink("FailingOutput", new DataSinkDescriptor(od, ocd, null));
+ dag.getVertex("v3").addDataSink("FailingOutput", DataSinkDescriptor.create(od, ocd, null));
runDAGAndVerify(dag, State.FAILED);
}