From 4563e7af03b2709f154fd296c368cd9075760db4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Hank=C3=B3=20Gergely?= Date: Tue, 23 Nov 2021 15:32:02 +0100 Subject: [PATCH] TEZ-4227 Introduce convenient methods in TezID subclasses Change-Id: I6cabfa75e9b6b62e41ba8c2cc5e3d2d1a8a49102 --- .../apache/tez/dag/records/DAGIDAware.java | 29 ++ .../tez/dag/records/TaskAttemptIDAware.java | 28 ++ .../apache/tez/dag/records/TaskIDAware.java | 28 ++ .../tez/dag/records/TezTaskAttemptID.java | 3 +- .../org/apache/tez/dag/records/TezTaskID.java | 3 +- .../apache/tez/dag/records/TezVertexID.java | 6 +- .../apache/tez/dag/records/VertexIDAware.java | 28 ++ .../tez/dag/records/VertexIdentifierImpl.java | 2 +- .../apache/tez/dag/records/TestTezIds.java | 2 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 17 +- .../apache/tez/dag/app/RecoveryParser.java | 12 +- .../tez/dag/app/TaskCommunicatorManager.java | 6 +- .../apache/tez/dag/app/dag/DAGScheduler.java | 8 +- .../app/dag/RootInputInitializerManager.java | 6 +- .../java/org/apache/tez/dag/app/dag/Task.java | 5 +- .../apache/tez/dag/app/dag/TaskAttempt.java | 11 +- .../tez/dag/app/dag/event/DAGEvent.java | 6 +- .../dag/event/DAGEventSchedulerUpdate.java | 13 +- .../dag/event/DAGEventVertexCompleted.java | 4 +- .../dag/event/DAGEventVertexReRunning.java | 2 +- ...peculatorEventTaskAttemptStatusUpdate.java | 2 +- .../dag/app/dag/event/TaskAttemptEvent.java | 4 +- .../tez/dag/app/dag/event/TaskEvent.java | 4 +- .../tez/dag/app/dag/event/VertexEvent.java | 6 +- .../VertexEventTaskAttemptCompleted.java | 2 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 4 +- .../dag/impl/DAGSchedulerNaturalOrder.java | 6 +- .../DAGSchedulerNaturalOrderControlled.java | 15 +- .../org/apache/tez/dag/app/dag/impl/Edge.java | 2 +- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 62 ++-- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 88 ++--- .../TezRootInputInitializerContextImpl.java | 2 +- .../tez/dag/app/dag/impl/VertexImpl.java | 26 +- .../tez/dag/app/dag/impl/VertexManager.java | 6 +- .../speculation/legacy/LegacySpeculator.java | 8 +- .../TaskAttemptFailedDeleteRunnable.java | 4 +- .../dag/app/rm/AMSchedulerEventTAEnded.java | 2 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 28 +- .../AMContainerEventLaunchRequest.java | 2 +- .../tez/dag/app/web/AMWebController.java | 6 +- .../tez/dag/history/DAGHistoryEvent.java | 6 +- .../tez/dag/history/HistoryEventHandler.java | 6 +- .../dag/history/events/DAGFinishedEvent.java | 6 +- .../history/events/DAGInitializedEvent.java | 6 +- .../dag/history/events/DAGStartedEvent.java | 6 +- .../dag/history/events/DAGSubmittedEvent.java | 6 +- .../events/TaskAttemptFinishedEvent.java | 4 +- .../events/TaskAttemptStartedEvent.java | 4 +- .../dag/history/events/TaskFinishedEvent.java | 4 +- .../dag/history/events/TaskStartedEvent.java | 4 +- .../events/VertexCommitStartedEvent.java | 2 +- .../events/VertexConfigurationDoneEvent.java | 4 +- .../history/events/VertexFinishedEvent.java | 6 +- .../events/VertexInitializedEvent.java | 6 +- .../history/events/VertexStartedEvent.java | 6 +- .../impl/HistoryEventJsonConversion.java | 16 +- .../dag/history/recovery/RecoveryService.java | 12 +- .../tez/dag/app/TestMockDAGAppMaster.java | 12 +- .../apache/tez/dag/app/TestSpeculation.java | 10 +- .../tez/dag/app/dag/impl/TestCommit.java | 136 ++++---- .../tez/dag/app/dag/impl/TestDAGImpl.java | 40 +-- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 7 +- .../dag/app/dag/impl/TestDAGScheduler.java | 41 ++- ...estDAGSchedulerNaturalOrderControlled.java | 6 +- .../apache/tez/dag/app/dag/impl/TestEdge.java | 6 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 73 +++-- .../tez/dag/app/dag/impl/TestTaskImpl.java | 300 +++++++++--------- .../tez/dag/app/dag/impl/TestVertexImpl.java | 44 +-- .../tez/dag/app/rm/TestContainerReuse.java | 12 +- .../dag/app/rm/TestTaskSchedulerManager.java | 6 +- .../tez/dag/app/web/TestAMWebController.java | 14 +- .../TestHistoryEventsProtoConversion.java | 20 +- .../tez/service/impl/ContainerRunnerImpl.java | 4 +- .../tez/mapreduce/hadoop/IDConverter.java | 2 +- .../mapreduce/TaskAttemptContextImpl.java | 6 +- .../tez/mapreduce/processor/MRTask.java | 4 +- .../history/parser/SimpleHistoryParser.java | 12 +- .../history/parser/datamodel/BaseParser.java | 5 +- .../proto/HistoryEventProtoConverter.java | 16 +- .../proto/ProtoHistoryLoggingService.java | 12 +- .../logging/ats/TimelineCachePluginImpl.java | 6 +- .../ats/acls/TestATSHistoryWithACLs.java | 4 +- .../ats/ATSV15HistoryLoggingService.java | 12 +- .../logging/ats/ATSHistoryLoggingService.java | 8 +- .../ats/HistoryEventTimelineConversion.java | 81 +++-- .../tez/runtime/api/impl/EventMetaData.java | 4 +- .../apache/tez/runtime/api/impl/TaskSpec.java | 7 +- .../runtime/api/impl/TezTaskContextImpl.java | 7 +- .../org/apache/tez/runtime/task/TezChild.java | 6 +- .../RecoveryServiceWithEventHandlingHook.java | 16 +- .../org/apache/tez/test/TestAMRecovery.java | 4 +- 91 files changed, 834 insertions(+), 711 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java new file mode 100644 index 0000000000..1234a30053 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +import org.apache.hadoop.yarn.api.records.ApplicationId; + +public interface DAGIDAware { + TezDAGID getDAGID(); + + default ApplicationId getApplicationId() { + return getDAGID().getApplicationId(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java new file mode 100644 index 0000000000..924fd07109 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface TaskAttemptIDAware extends TaskIDAware { + TezTaskAttemptID getTaskAttemptID(); + + @Override + default TezTaskID getTaskID() { + return getTaskAttemptID().getTaskID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java new file mode 100644 index 0000000000..0bee45dfa1 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface TaskIDAware extends VertexIDAware { + TezTaskID getTaskID(); + + @Override + default TezVertexID getVertexID() { + return getTaskID().getVertexID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java index 7c92be9449..fe2b84449f 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java @@ -46,7 +46,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezTaskAttemptID extends TezID { +public class TezTaskAttemptID extends TezID implements TaskIDAware { public static final String ATTEMPT = "attempt"; private TezTaskID taskId; @@ -73,6 +73,7 @@ private TezTaskAttemptID(TezTaskID taskId, int id) { } /** Returns the {@link TezTaskID} object that this task attempt belongs to */ + @Override public TezTaskID getTaskID() { return taskId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index a1a556c9ba..08310f3dfc 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -41,7 +41,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezTaskID extends TezID { +public class TezTaskID extends TezID implements VertexIDAware { public static final String TASK = "task"; private final int serializingHash; @@ -79,6 +79,7 @@ public int getSerializingHash() { } /** Returns the {@link TezVertexID} object that this task belongs to */ + @Override public TezVertexID getVertexID() { return vertexId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index e9ddd77d38..f7becc250f 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -44,7 +44,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezVertexID extends TezID { +public class TezVertexID extends TezID implements DAGIDAware { public static final String VERTEX = "vertex"; static final ThreadLocal tezVertexIdFormat = new ThreadLocal() { @@ -80,7 +80,8 @@ private TezVertexID(TezDAGID dagId, int id) { } /** Returns the {@link TezDAGID} object that this tip belongs to */ - public TezDAGID getDAGId() { + @Override + public TezDAGID getDAGID() { return dagId; } @@ -159,5 +160,4 @@ public static TezVertexID fromString(String vertexIdStr) { } return null; } - } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java new file mode 100644 index 0000000000..01bbe859b6 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface VertexIDAware extends DAGIDAware { + TezVertexID getVertexID(); + + @Override + default TezDAGID getDAGID() { + return getVertexID().getDAGID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java index 4480f742fc..83b503203c 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java @@ -30,7 +30,7 @@ public class VertexIdentifierImpl implements VertexIdentifier { public VertexIdentifierImpl(String dagName, String vertexName, TezVertexID vertexId) { this.vertexId = vertexId; this.vertexName = vertexName; - this.dagIdentifier = new DagIdentifierImpl(dagName, vertexId.getDAGId()); + this.dagIdentifier = new DagIdentifierImpl(dagName, vertexId.getDAGID()); } @Override diff --git a/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java b/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java index 5e1552d345..10b62b2bd1 100644 --- a/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java +++ b/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java @@ -38,7 +38,7 @@ private void verifyDagInfo(String[] splits, TezDAGID dagId) { } private void verifyVertexInfo(String[] splits, TezVertexID vId) { - verifyDagInfo(splits, vId.getDAGId()); + verifyDagInfo(splits, vId.getDAGID()); Assert.assertEquals(vId.getId(), Integer.valueOf(splits[4]).intValue()); } 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 3be9aacec5..972fadf854 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 @@ -50,7 +50,6 @@ import java.util.TimerTask; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -2173,7 +2172,7 @@ private class DagEventDispatcher implements EventHandler { @Override public void handle(DAGEvent event) { DAG dag = context.getCurrentDAG(); - int eventDagIndex = event.getDAGId().getId(); + int eventDagIndex = event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } @@ -2187,12 +2186,12 @@ private class TaskEventDispatcher implements EventHandler { public void handle(TaskEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getTaskID().getVertexID().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Task task = - dag.getVertex(event.getTaskID().getVertexID()). + dag.getVertex(event.getVertexID()). getTask(event.getTaskID()); ((EventHandler)task).handle(event); } @@ -2217,13 +2216,13 @@ private class TaskAttemptEventDispatcher public void handle(TaskAttemptEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Task task = - dag.getVertex(event.getTaskAttemptID().getTaskID().getVertexID()). - getTask(event.getTaskAttemptID().getTaskID()); + dag.getVertex(event.getVertexID()). + getTask(event.getTaskID()); TaskAttempt attempt = task.getAttempt(event.getTaskAttemptID()); ((EventHandler) attempt).handle(event); } @@ -2236,13 +2235,13 @@ private class VertexEventDispatcher public void handle(VertexEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getVertexId().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Vertex vertex = - dag.getVertex(event.getVertexId()); + dag.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } 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 19c24f300c..ce3b62bbd5 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 @@ -846,9 +846,9 @@ public DAGRecoveryData parseRecoveryData() throws IOException { case TASK_STARTED: { TaskStartedEvent taskStartedEvent = (TaskStartedEvent) event; - VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskStartedEvent.getTaskID().getVertexID()); + VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskStartedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, - "Invalid TaskStartedEvent, its vertex does not exist:" + taskStartedEvent.getTaskID().getVertexID()); + "Invalid TaskStartedEvent, its vertex does not exist:" + taskStartedEvent.getVertexID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskStartedEvent.getTaskID()); taskRecoveryData.taskStartedEvent = taskStartedEvent; break; @@ -856,9 +856,9 @@ public DAGRecoveryData parseRecoveryData() throws IOException { case TASK_FINISHED: { TaskFinishedEvent taskFinishedEvent = (TaskFinishedEvent) event; - VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskFinishedEvent.getTaskID().getVertexID()); + VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskFinishedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, - "Invalid TaskFinishedEvent, its vertex does not exist:" + taskFinishedEvent.getTaskID().getVertexID()); + "Invalid TaskFinishedEvent, its vertex does not exist:" + taskFinishedEvent.getVertexID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskFinishedEvent.getTaskID()); taskRecoveryData.taskFinishedEvent = taskFinishedEvent; break; @@ -867,7 +867,7 @@ public DAGRecoveryData parseRecoveryData() throws IOException { { TaskAttemptStartedEvent taStartedEvent = (TaskAttemptStartedEvent)event; VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get( - taStartedEvent.getTaskAttemptID().getTaskID().getVertexID()); + taStartedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, "Invalid TaskAttemptStartedEvent, its vertexId does not exist, taId=" + taStartedEvent.getTaskAttemptID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap @@ -882,7 +882,7 @@ public DAGRecoveryData parseRecoveryData() throws IOException { { TaskAttemptFinishedEvent taFinishedEvent = (TaskAttemptFinishedEvent)event; VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get( - taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID()); + taFinishedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, "Invalid TaskAttemtFinishedEvent, its vertexId does not exist, taId=" + taFinishedEvent.getTaskAttemptID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index ac2f7605ae..51895f4afd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -350,14 +350,14 @@ public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) } } if (!eventsForVertex.isEmpty()) { - TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID(); + TezVertexID vertexId = taskAttemptID.getVertexID(); sendEvent( new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(eventsForVertex))); } taskHeartbeatHandler.pinged(taskAttemptID); eventInfo = context .getCurrentDAG() - .getVertex(taskAttemptID.getTaskID().getVertexID()) + .getVertex(taskAttemptID.getVertexID()) .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(), request.getPreRoutedStartIndex(), request.getMaxEvents()); } @@ -442,7 +442,7 @@ public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException { DAG job = context.getCurrentDAG(); Task task = - job.getVertex(taskAttemptId.getTaskID().getVertexID()). + job.getVertex(taskAttemptId.getVertexID()). getTask(taskAttemptId.getTaskID()); return task.canCommit(taskAttemptId); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java index 2fa735ef8f..ada7867a6a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java @@ -52,7 +52,7 @@ public void addVertexConcurrencyLimit(TezVertexID vId, int concurrency) { public void scheduleTask(DAGEventSchedulerUpdate event) { VertexInfo vInfo = null; if (vertexInfo != null) { - vInfo = vertexInfo.get(event.getAttempt().getID().getTaskID().getVertexID()); + vInfo = vertexInfo.get(event.getVertexID()); } scheduleTaskWithLimit(event, vInfo); } @@ -60,7 +60,7 @@ public void scheduleTask(DAGEventSchedulerUpdate event) { private void scheduleTaskWithLimit(DAGEventSchedulerUpdate event, VertexInfo vInfo) { if (vInfo != null) { if (vInfo.concurrency >= vInfo.concurrencyLimit) { - vInfo.pendingAttempts.put(event.getAttempt().getID(), event); + vInfo.pendingAttempts.put(event.getTaskAttemptID(), event); return; // already at max concurrency } vInfo.concurrency++; @@ -71,9 +71,9 @@ private void scheduleTaskWithLimit(DAGEventSchedulerUpdate event, VertexInfo vIn public void taskCompleted(DAGEventSchedulerUpdate event) { taskCompletedEx(event); if (vertexInfo != null) { - VertexInfo vInfo = vertexInfo.get(event.getAttempt().getID().getTaskID().getVertexID()); + VertexInfo vInfo = vertexInfo.get(event.getVertexID()); if (vInfo != null) { - if(vInfo.pendingAttempts.remove(event.getAttempt().getID()) == null) { + if(vInfo.pendingAttempts.remove(event.getTaskAttemptID()) == null) { vInfo.concurrency--; if(!vInfo.pendingAttempts.isEmpty()) { Iterator i = vInfo.pendingAttempts.values().iterator(); 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 3c4a05eb84..cfbdb19e30 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 @@ -457,7 +457,7 @@ public void onTaskSucceeded(String vertexName, TezTaskID taskId, int attemptId) Iterator eventIterator = events.iterator(); while (eventIterator.hasNext()) { TezEvent tezEvent = eventIterator.next(); - int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId(); + int taskIndex = tezEvent.getSourceInfo().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 @@ -476,7 +476,7 @@ public void handleInputInitializerEvents(Collection tezEvents) { List toForwardEvents = new LinkedList(); for (TezEvent tezEvent : tezEvents) { String srcVertexName = tezEvent.getSourceInfo().getTaskVertexName(); - int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId(); + int taskIndex = tezEvent.getSourceInfo().getTaskID().getId(); int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId(); Map vertexSuccessfulAttemptMap = @@ -496,7 +496,7 @@ public void handleInputInitializerEvents(Collection tezEvents) { Vertex srcVertex = appContext.getCurrentDAG().getVertex(srcVertexName); Task task = srcVertex.getTask(taskIndex); if (task.getState() == TaskState.SUCCEEDED) { - successfulAttemptInteger = task.getSuccessfulAttempt().getID().getId(); + successfulAttemptInteger = task.getSuccessfulAttempt().getTaskAttemptID().getId(); vertexSuccessfulAttemptMap.put(taskIndex, successfulAttemptInteger); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java index c1fe7f79ad..cb1bc33866 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java @@ -28,16 +28,15 @@ import org.apache.tez.dag.api.TaskLocationHint; import org.apache.tez.dag.api.oldrecords.TaskReport; import org.apache.tez.dag.api.oldrecords.TaskState; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.runtime.api.impl.TaskSpec; import org.apache.tez.runtime.api.impl.TezEvent; /** * Read only view of Task. */ -public interface Task { - TezTaskID getTaskId(); +public interface Task extends TaskIDAware { TaskReport getReport(); TaskState getState(); TezCounters getCounters(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java index 0fc7013fea..563e4c7192 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java @@ -28,17 +28,15 @@ import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.oldrecords.TaskAttemptReport; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TaskAttemptTerminationCause; -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.impl.TezEvent; /** * Read only view of TaskAttempt. */ -public interface TaskAttempt { +public interface TaskAttempt extends TaskAttemptIDAware { public static class TaskAttemptStatus { public TezTaskAttemptID id; @@ -66,11 +64,6 @@ public void setLocalityCounter(DAGCounter localityCounter) { } } } - - TezTaskAttemptID getID(); - TezTaskID getTaskID(); - TezVertexID getVertexID(); - TezDAGID getDAGID(); Task getTask(); TaskAttemptReport getReport(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java index a0a8a1af8e..4eed8ff984 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; /** * This class encapsulates job related events. * */ -public class DAGEvent extends TezAbstractEvent { +public class DAGEvent extends TezAbstractEvent implements DAGIDAware { private TezDAGID dagId; @@ -34,7 +35,8 @@ public DAGEvent(TezDAGID dagId, DAGEventType type) { this.dagId = dagId; } - public TezDAGID getDAGId() { + @Override + public TezDAGID getDAGID() { return dagId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java index eda02b52da..1dedaecad9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java @@ -19,9 +19,11 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.records.TaskAttemptIDAware; +import org.apache.tez.dag.records.TezTaskAttemptID; + +public class DAGEventSchedulerUpdate extends DAGEvent implements TaskAttemptIDAware { -public class DAGEventSchedulerUpdate extends DAGEvent { - public enum UpdateType { TA_SCHEDULE, TA_COMPLETED @@ -31,7 +33,7 @@ public enum UpdateType { private final UpdateType updateType; public DAGEventSchedulerUpdate(UpdateType updateType, TaskAttempt attempt) { - super(attempt.getDAGID(), + super(attempt.getDAGID(), DAGEventType.DAG_SCHEDULER_UPDATE); this.attempt = attempt; this.updateType = updateType; @@ -44,4 +46,9 @@ public UpdateType getUpdateType() { public TaskAttempt getAttempt() { return attempt; } + + @Override + public TezTaskAttemptID getTaskAttemptID() { + return attempt.getTaskAttemptID(); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java index e58d46e834..2c18be85dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java @@ -29,7 +29,7 @@ public class DAGEventVertexCompleted extends DAGEvent { private final VertexTerminationCause terminationCause; public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_COMPLETED); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_COMPLETED); this.vertexId = vertexId; this.vertexState = vertexState; this.terminationCause = null; @@ -37,7 +37,7 @@ public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState) { public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState, VertexTerminationCause terminationCause) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_COMPLETED); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_COMPLETED); this.vertexId = vertexId; this.vertexState = vertexState; this.terminationCause = terminationCause; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java index 303d48d375..30e595906b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java @@ -25,7 +25,7 @@ public class DAGEventVertexReRunning extends DAGEvent { private TezVertexID vertexId; public DAGEventVertexReRunning(TezVertexID vertexId) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_RERUNNING); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_RERUNNING); this.vertexId = vertexId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java index d5745c4df4..7ab6141c39 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java @@ -34,7 +34,7 @@ public SpeculatorEventTaskAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttempt public SpeculatorEventTaskAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttemptState state, long timestamp, boolean justStarted) { - super(SpeculatorEventType.S_TASK_ATTEMPT_STATUS_UPDATE, taId.getTaskID().getVertexID()); + super(SpeculatorEventType.S_TASK_ATTEMPT_STATUS_UPDATE, taId.getVertexID()); this.id = taId; this.state = state; this.timestamp = timestamp; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java index 63ef70feb4..d5d8481a54 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; /** * This class encapsulates task attempt related events. * */ -public class TaskAttemptEvent extends TezAbstractEvent { +public class TaskAttemptEvent extends TezAbstractEvent implements TaskAttemptIDAware { private TezTaskAttemptID attemptID; @@ -39,6 +40,7 @@ public TaskAttemptEvent(TezTaskAttemptID id, TaskAttemptEventType type) { this.attemptID = id; } + @Override public TezTaskAttemptID getTaskAttemptID() { return attemptID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java index def9ddfa7c..9e741388fa 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskID; /** * this class encapsulates task related events. * */ -public class TaskEvent extends TezAbstractEvent { +public class TaskEvent extends TezAbstractEvent implements TaskIDAware { private TezTaskID taskId; @@ -34,6 +35,7 @@ public TaskEvent(TezTaskID taskId, TaskEventType type) { this.taskId = taskId; } + @Override public TezTaskID getTaskID() { return taskId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java index 33128e4536..6957a50dc7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java @@ -20,12 +20,13 @@ import org.apache.tez.common.TezAbstractEvent; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; /** * this class encapsulates vertex related events. * */ -public class VertexEvent extends TezAbstractEvent { +public class VertexEvent extends TezAbstractEvent implements VertexIDAware { private TezVertexID vertexId; @@ -34,7 +35,8 @@ public VertexEvent(TezVertexID vertexId, VertexEventType type) { this.vertexId = vertexId; } - public TezVertexID getVertexId() { + @Override + public TezVertexID getVertexID() { return vertexId; } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java index 5b07674422..e23d27cf6e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java @@ -28,7 +28,7 @@ public class VertexEventTaskAttemptCompleted extends VertexEvent { public VertexEventTaskAttemptCompleted(TezTaskAttemptID taskAttemptId, TaskAttemptStateInternal state) { - super(taskAttemptId.getTaskID().getVertexID(), + super(taskAttemptId.getVertexID(), VertexEventType.V_TASK_ATTEMPT_COMPLETED); this.attemptId = taskAttemptId; this.attempState = state; 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 07715cdfe7..c9337e473a 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 @@ -1040,7 +1040,7 @@ public VertexStatusBuilder getVertexStatus(String vertexName, } public TaskAttemptImpl getTaskAttempt(TezTaskAttemptID taId) { - return (TaskAttemptImpl) getVertex(taId.getTaskID().getVertexID()).getTask(taId.getTaskID()) + return (TaskAttemptImpl) getVertex(taId.getVertexID()).getTask(taId.getTaskID()) .getAttempt(taId); } @@ -1206,7 +1206,7 @@ private void abortOutputs() { */ public void handle(DAGEvent event) { if (LOG.isDebugEnabled()) { - LOG.debug("Processing DAGEvent " + event.getDAGId() + " of type " + LOG.debug("Processing DAGEvent " + event.getDAGID() + " of type " + event.getType() + " while in state " + getInternalState() + ". Event: " + event); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java index 2383db83b1..78860868dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -52,12 +52,12 @@ public void scheduleTaskEx(DAGEventSchedulerUpdate event) { int priorityHighLimit = getPriorityHighLimit(dag, vertex); if (LOG.isDebugEnabled()) { - LOG.debug("Scheduling " + attempt.getID() + " between priorityLow: " + priorityLowLimit + LOG.debug("Scheduling " + attempt.getTaskAttemptID() + " between priorityLow: " + priorityLowLimit + " and priorityHigh: " + priorityHighLimit); } TaskAttemptEventSchedule attemptEvent = new TaskAttemptEventSchedule( - attempt.getID(), priorityLowLimit, priorityHighLimit); + attempt.getTaskAttemptID(), priorityLowLimit, priorityHighLimit); sendEvent(attemptEvent); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java index c51783bc2d..3167c9f3b7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -62,9 +62,9 @@ public class DAGSchedulerNaturalOrderControlled extends DAGScheduler { LinkedListMultimap.create(); // Tacks vertices for which no additional scheduling checks are required. Once in this list, the // vertex is considered to be fully scheduled. - private final Set scheduledVertices = new HashSet(); + private final Set scheduledVertices = new HashSet<>(); // Tracks tasks scheduled for a vertex. - private final Map vertexScheduledTasks = new HashMap(); + private final Map vertexScheduledTasks = new HashMap<>(); public DAGSchedulerNaturalOrderControlled(DAG dag, EventHandler dispatcher) { this.dag = dag; @@ -82,14 +82,14 @@ public void scheduleTaskEx(DAGEventSchedulerUpdate event) { int priorityHighLimit = getPriorityHighLimit(dag, vertex); TaskAttemptEventSchedule attemptEvent = new TaskAttemptEventSchedule( - attempt.getID(), priorityLowLimit, priorityHighLimit); + attempt.getTaskAttemptID(), priorityLowLimit, priorityHighLimit); - taskAttemptSeen(vertex.getName(), attempt.getID()); + taskAttemptSeen(vertex.getName(), attempt.getTaskAttemptID()); if (vertexAlreadyScheduled(vertex)) { // Vertex previously marked ready for scheduling. if (LOG.isDebugEnabled()) { - LOG.debug("Scheduling " + attempt.getID() + " between priorityLow: " + priorityLowLimit + LOG.debug("Scheduling " + attempt.getTaskAttemptID() + " between priorityLow: " + priorityLowLimit + " and priorityHigh: " + priorityHighLimit); } sendEvent(attemptEvent); @@ -154,8 +154,7 @@ private void processDownstreamVertices(Vertex vertex) { List newlyScheduledVertices = Lists.newLinkedList(); Map outputVertexEdgeMap = vertex.getOutputVertices(); for (Vertex destVertex : outputVertexEdgeMap.keySet()) { - if (vertexAlreadyScheduled(destVertex)) { // Nothing to do if already scheduled. - } else { + if (!vertexAlreadyScheduled(destVertex)) { if (LOG.isDebugEnabled()) { LOG.debug("Attempting to schedule vertex: " + destVertex.getLogIdentifier() + " due to upstream event from " + vertex.getLogIdentifier()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java index 99b56fbf07..08e1c19cee 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java @@ -413,7 +413,7 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept " destIndex=" + destTaskIndex + " edgeManager=" + edgeManager.getClass().getName()); } - TezTaskID srcTaskId = srcTask.getTaskId(); + TezTaskID srcTaskId = srcTask.getTaskID(); int srcTaskAttemptIndex = inputReadErrorEvent.getVersion(); TezTaskAttemptID srcTaskAttemptId = TezTaskAttemptID.getInstance(srcTaskId, srcTaskAttemptIndex); 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 e6b9e82000..c8343c834b 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 @@ -104,10 +104,7 @@ import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent; import org.apache.tez.dag.history.events.TaskAttemptStartedEvent; import org.apache.tez.dag.records.TaskAttemptTerminationCause; -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.dag.recovery.records.RecoveryProtos.DataEventDependencyInfoProto; import org.apache.tez.runtime.api.events.InputFailedEvent; import org.apache.tez.runtime.api.events.InputReadErrorEvent; @@ -588,25 +585,10 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, } @Override - public TezTaskAttemptID getID() { + public TezTaskAttemptID getTaskAttemptID() { return attemptId; } - @Override - public TezTaskID getTaskID() { - return attemptId.getTaskID(); - } - - @Override - public TezVertexID getVertexID() { - return attemptId.getTaskID().getVertexID(); - } - - @Override - public TezDAGID getDAGID() { - return getVertexID().getDAGId(); - } - public TezTaskAttemptID getSchedulingCausalTA() { return creationCausalTA; } @@ -884,12 +866,12 @@ public void handle(TaskAttemptEvent event) { LOG.error("Can't handle this event at current state for " + this.attemptId, e); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), "Invalid event " + event.getType() + " on TaskAttempt " + this.attemptId)); eventHandler.handle( new DAGEvent( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), DAGEventType.INTERNAL_ERROR) ); } catch (RuntimeException e) { @@ -897,13 +879,13 @@ public void handle(TaskAttemptEvent event) { + " at current state " + oldState + " for " + this.attemptId, e); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), "Uncaught exception when handling event " + event.getType() + " on TaskAttempt " + this.attemptId + " at state " + oldState + ", error=" + e.getMessage())); eventHandler.handle( new DAGEvent( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), DAGEventType.INTERNAL_ERROR) ); } @@ -1269,7 +1251,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent if (taFinishedEvent == null) { LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, " + "send out TaskAttemptEventAttemptKilled to move it to KILLED"); - ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), + ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getTaskAttemptID(), "Task Attempt killed in recovery due to can't recover the running task attempt", TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, true)); return TaskAttemptStateInternal.NEW; @@ -1280,29 +1262,29 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData.getTaskAttemptFinishedEvent(); Preconditions.checkArgument(taFinishedEvent != null, "Both of TaskAttemptStartedEvent and TaskFinishedEvent is null," - + "taskAttemptId=" + ta.getID()); + + "taskAttemptId=" + ta.getTaskAttemptID()); switch (taFinishedEvent.getState()) { case FAILED: LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED, " + "send TA_FAILED to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, taFinishedEvent.getTaskFailureType(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case KILLED: LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED, " + "send TA_KILLED to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), + ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getTaskAttemptID(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case SUCCEEDED: LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED, " + "send TA_DONE to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEvent(ta.getID(), TaskAttemptEventType.TA_DONE)); + ta.sendEvent(new TaskAttemptEvent(ta.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); break; default: throw new TezUncheckedException("Invalid state in TaskAttemptFinishedEvent, state=" - + taFinishedEvent.getState() + ", taId=" + ta.getID()); + + taFinishedEvent.getState() + ", taId=" + ta.getTaskAttemptID()); } return TaskAttemptStateInternal.NEW; } @@ -1428,7 +1410,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { RecoveryEvent rEvent = (RecoveryEvent)event; if (rEvent.isFromRecovery()) { if (LOG.isDebugEnabled()) { - LOG.debug("Faked TerminateEvent from recovery, taskAttemptId=" + ta.getID()); + LOG.debug("Faked TerminateEvent from recovery, taskAttemptId=" + ta.getTaskAttemptID()); } } } @@ -1613,10 +1595,10 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { // task is hung String diagnostics = "Attempt failed because it appears to make no progress for " + ta.hungIntervalMax + "ms"; - LOG.info(diagnostics + " " + ta.getID()); + LOG.info(diagnostics + " " + ta.getTaskAttemptID()); // send event that will fail this attempt ta.sendEvent( - new TaskAttemptEventAttemptFailed(ta.getID(), + new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, diagnostics, @@ -1803,8 +1785,8 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)inputFailedEvent.getEvent(); int failedInputIndexOnDestTa = readErrorEvent.getIndex(); - if (readErrorEvent.getVersion() != sourceAttempt.getID().getId()) { - throw new TezUncheckedException(sourceAttempt.getID() + if (readErrorEvent.getVersion() != sourceAttempt.getTaskAttemptID().getId()) { + throw new TezUncheckedException(sourceAttempt.getTaskAttemptID() + " incorrectly blamed for read error from " + failedDestTaId + " at inputIndex " + failedInputIndexOnDestTa + " version" + readErrorEvent.getVersion()); @@ -1814,7 +1796,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, // destination: where the data is tried to be fetched to String dHost = readErrorEvent.getDestinationLocalhostName(); - LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getID(), + LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getTaskAttemptID(), sHost, failedDestTaId, dHost, failedInputIndexOnDestTa); boolean tooManyDownstreamHostsBlamedTheSameUpstreamHost = false; @@ -1856,7 +1838,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, boolean crossTimeDeadline = readErrorTimespanSec >= maxAllowedTimeForTaskReadErrorSec; int runningTasks = sourceAttempt.appContext.getCurrentDAG().getVertex( - failedDestTaId.getTaskID().getVertexID()).getRunningTasks(); + failedDestTaId.getVertexID()).getRunningTasks(); float failureFraction = runningTasks > 0 ? ((float) sourceAttempt.uniquefailedOutputReports.size()) / runningTasks : 0; boolean withinFailureFractionLimits = @@ -1872,7 +1854,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, && !tooManyDownstreamHostsBlamedTheSameUpstreamHost) { return sourceAttempt.getInternalState(); } - String message = sourceAttempt.getID() + " being failed for too many output errors. " + String message = sourceAttempt.getTaskAttemptID() + " being failed for too many output errors. " + "failureFraction=" + failureFraction + ", MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=" + maxAllowedOutputFailuresFraction @@ -1929,8 +1911,8 @@ protected void sendInputFailedToConsumers() { tezIfEvents.add(new TezEvent(new InputFailedEvent(), new EventMetaData(EventProducerConsumerType.SYSTEM, vertex.getName(), - edgeVertex.getName(), - getID()), appContext.getClock().getTime())); + edgeVertex.getName(), + getTaskAttemptID()), appContext.getClock().getTime())); } sendEvent(new VertexEventRouteEvent(vertex.getVertexId(), tezIfEvents)); } @@ -2024,7 +2006,7 @@ public TaskFailureType getFailureType(TaskAttemptEvent event) { @Override public String toString() { - return getID().toString(); + return getTaskAttemptID().toString(); } 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 b840798c26..d2933c5b86 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 @@ -40,6 +40,10 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; +import org.apache.tez.dag.records.TaskAttemptTerminationCause; +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.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,10 +97,6 @@ import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.events.TaskFinishedEvent; import org.apache.tez.dag.history.events.TaskStartedEvent; -import org.apache.tez.dag.records.TaskAttemptTerminationCause; -import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.TezBuilderUtils; import org.apache.tez.runtime.api.OutputCommitter; import org.apache.tez.runtime.api.impl.TaskSpec; @@ -434,7 +434,7 @@ public Vertex getVertex() { } @Override - public TezTaskID getTaskId() { + public TezTaskID getTaskID() { return taskId; } @@ -523,7 +523,7 @@ public ArrayList getTaskAttemptTezEvents(TezTaskAttemptID attemptID, try { if (!attempts.containsKey(attemptID)) { throw new TezUncheckedException("Unknown TA: " + attemptID - + " asking for events from task:" + getTaskId()); + + " asking for events from task:" + getTaskID()); } if (tezEventsForTaskAttempts.size() > fromEventId) { @@ -775,11 +775,11 @@ public TaskAttempt getSuccessfulAttempt() { private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { TaskAttempt attempt = createAttempt(attempts.size(), schedulingCausalTA); if (LOG.isDebugEnabled()) { - LOG.debug("Created attempt " + attempt.getID()); + LOG.debug("Created attempt " + attempt.getTaskAttemptID()); } switch (attempts.size()) { case 0: - attempts = Collections.singletonMap(attempt.getID(), attempt); + attempts = Collections.singletonMap(attempt.getTaskAttemptID(), attempt); break; case 1: @@ -787,12 +787,12 @@ private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { = new LinkedHashMap(maxFailedAttempts); newAttempts.putAll(attempts); attempts = newAttempts; - Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, - attempt.getID() + " already existed"); + Preconditions.checkArgument(attempts.put(attempt.getTaskAttemptID(), attempt) == null, + attempt.getTaskAttemptID() + " already existed"); break; default: - Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, - attempt.getID() + " already existed"); + Preconditions.checkArgument(attempts.put(attempt.getTaskAttemptID(), attempt) == null, + attempt.getTaskAttemptID() + " already existed"); break; } @@ -819,7 +819,7 @@ private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { } */ - this.taskAttemptStatus.put(attempt.getID().getId(), false); + this.taskAttemptStatus.put(attempt.getTaskAttemptID().getId(), false); //schedule the nextAttemptNumber // send event to DAG to assign priority and schedule the attempt with global // picture in mind @@ -865,17 +865,17 @@ protected void internalError(TaskEventType type) { LOG.error("Invalid event " + type + " on Task " + this.taskId + " in state:" + getInternalState()); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.taskId.getVertexID().getDAGId(), "Invalid event " + type + + getDAGID(), "Invalid event " + type + " on Task " + this.taskId)); - eventHandler.handle(new DAGEvent(this.taskId.getVertexID().getDAGId(), + eventHandler.handle(new DAGEvent(getDAGID(), DAGEventType.INTERNAL_ERROR)); } protected void internalErrorUncaughtException(TaskEventType type, Exception e) { eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.taskId.getVertexID().getDAGId(), "Uncaught exception when handling event " + type + + getDAGID(), "Uncaught exception when handling event " + type + " on Task " + this.taskId + ", error=" + e.getMessage())); - eventHandler.handle(new DAGEvent(this.taskId.getVertexID().getDAGId(), + eventHandler.handle(new DAGEvent(getDAGID(), DAGEventType.INTERNAL_ERROR)); } @@ -918,7 +918,7 @@ protected void logJobHistoryTaskStartedEvent() { TaskStartedEvent startEvt = new TaskStartedEvent(taskId, getVertex().getName(), scheduledTime, getLaunchTime()); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), startEvt)); + new DAGHistoryEvent(getDAGID(), startEvt)); } protected void logJobHistoryTaskFinishedEvent() { @@ -930,7 +930,7 @@ protected void logJobHistoryTaskFinishedEvent() { successfulAttempt, TaskState.SUCCEEDED, "", getCounters(), failedAttempts); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt)); + new DAGHistoryEvent(getDAGID(), finishEvt)); } protected void logJobHistoryTaskFailedEvent(TaskState finalState) { @@ -941,7 +941,7 @@ protected void logJobHistoryTaskFailedEvent(TaskState finalState) { StringUtils.join(getDiagnostics(), LINE_SEPARATOR), getCounters(), failedAttempts); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt)); + new DAGHistoryEvent(getDAGID(), finishEvt)); } private void addDiagnosticInfo(String diag) { @@ -994,7 +994,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { && tFinishedEvent != null) { Preconditions.checkArgument(tFinishedEvent.getState() == TaskState.KILLED, "TaskStartedEvent is not seen, but TaskFinishedEvent is seen and with invalid state=" - + tFinishedEvent.getState() + ", taskId=" + task.getTaskId()); + + tFinishedEvent.getState() + ", taskId=" + task.getTaskID()); // TODO (TEZ-2938) // use tFinishedEvent.getTerminationCause after adding TaskTerminationCause to TaskFinishedEvent task.eventHandler.handle(new TaskEventTermination(task.taskId, @@ -1043,22 +1043,22 @@ public void transition(TaskImpl task, TaskEvent event) { } else { if (TaskAttemptState.SUCCEEDED.equals(ta.getState())) { LOG.info("Ignore speculation scheduling for task {} since it has succeeded with attempt {}.", - task.getTaskId(), ta.getID()); + task.getTaskID(), ta.getTaskAttemptID()); return; } } } if (earliestUnfinishedAttempt == null) { // no running (or SUCCEEDED) task attempt at this moment, no need to schedule speculative attempt either - LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId()); + LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskID()); return; } if (task.commitAttempt != null) { LOG.info("Ignore speculation scheduling for task {} since commit has started with commitAttempt {}.", - task.getTaskId(), task.commitAttempt); + task.getTaskID(), task.commitAttempt); return; } - task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); + task.addAndScheduleAttempt(earliestUnfinishedAttempt.getTaskAttemptID()); } } @@ -1075,26 +1075,26 @@ private String recoverSuccessTaskAttempt(TaskImpl task) { for (Entry entry : task.getVertex().getOutputCommitters().entrySet()) { LOG.info("Recovering data for task from previous DAG attempt" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", output=" + entry.getKey()); OutputCommitter committer = entry.getValue(); if (!committer.isTaskRecoverySupported()) { errorMsg = "Task recovery not supported by committer" + ", failing task attempt"; LOG.info(errorMsg - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", attemptId=" + task.successfulAttempt + ", output=" + entry.getKey()); break; } try { - committer.recoverTask(task.getTaskId().getId(), + committer.recoverTask(task.getTaskID().getId(), task.appContext.getApplicationAttemptId().getAttemptId()-1); } catch (Exception e) { errorMsg = "Task recovery failed by committer: " + ExceptionUtils.getStackTrace(e); LOG.warn("Task recovery failed by committer" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", attemptId=" + task.successfulAttempt + ", output=" + entry.getKey(), e); break; @@ -1114,7 +1114,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { String errorMsg = recoverSuccessTaskAttempt(task); if (errorMsg != null) { LOG.info("Can not recover the successful task attempt, schedule new task attempt," - + "taskId=" + task.getTaskId()); + + "taskId=" + task.getTaskID()); task.successfulAttempt = null; if (!task.addAndScheduleAttempt(successTaId)) { task.finished(TaskStateInternal.FAILED); @@ -1150,12 +1150,12 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { // issue kill to all other attempts for (TaskAttempt attempt : task.attempts.values()) { - if (!attempt.getID().equals(task.successfulAttempt) && + if (!attempt.getTaskAttemptID().equals(task.successfulAttempt) && // This is okay because it can only talk us out of sending a // TA_KILL message to an attempt that doesn't need one for // other reasons. !attempt.isFinished()) { - LOG.info("Issuing kill to other attempt " + attempt.getID() + " as attempt: " + + LOG.info("Issuing kill to other attempt " + attempt.getTaskAttemptID() + " as attempt: " + task.successfulAttempt + " has succeeded"); String diagnostics = null; TaskAttemptTerminationCause errCause = null; @@ -1169,7 +1169,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { errCause = TaskAttemptTerminationCause.TERMINATED_INEFFECTIVE_SPECULATION; } task.eventHandler.handle(new TaskAttemptEventKillRequest(attempt - .getID(), diagnostics, errCause)); + .getTaskAttemptID(), diagnostics, errCause)); } } return task.finished(TaskStateInternal.SUCCEEDED); @@ -1267,7 +1267,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { TaskAttempt taskAttempt = task.getAttempt(castEvent.getTaskAttemptID()); if (taskAttempt.getAssignedContainer() != null) { NodeId nodeId = taskAttempt.getAssignedContainer().getNodeId(); - task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getID(), nodeId); + task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getTaskAttemptID(), nodeId); } task.taskAttemptStatus.put(castEvent.getTaskAttemptID().getId(), true); @@ -1278,7 +1278,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { TaskAttemptStateInternal.FAILED); // we don't need a new event if we already have a spare if (task.shouldScheduleNewAttempt()) { - LOG.info("Scheduling new attempt for task: " + task.getTaskId() + LOG.info("Scheduling new attempt for task: " + task.getTaskID() + ", currentFailedAttempts: " + task.failedAttempts + ", maxFailedAttempts: " + task.maxFailedAttempts + ", maxAttempts: " + task.maxAttempts); if (!task.addAndScheduleAttempt(getSchedulingCausalTA())){ @@ -1289,11 +1289,11 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { if (castEvent.getTaskFailureType() == TaskFailureType.NON_FATAL) { LOG.info( "Failing task: {} due to too many failed attempts. currentFailedAttempts={}, maxFailedAttempts={}", - task.getTaskId(), task.failedAttempts, task.maxFailedAttempts); + task.getTaskID(), task.failedAttempts, task.maxFailedAttempts); } else { LOG.info( "Failing task: {} due to {} error reported by TaskAttempt. CurrentFailedAttempts={}", - task.getTaskId(), TaskFailureType.FATAL, task.failedAttempts); + task.getTaskID(), TaskFailureType.FATAL, task.failedAttempts); } task.handleTaskAttemptCompletion( ((TaskEventTAUpdate) event).getTaskAttemptID(), @@ -1348,7 +1348,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { if (task.leafVertex) { LOG.error("Unexpected event for task of leaf vertex " + event.getType() + ", taskId: " - + task.getTaskId()); + + task.getTaskID()); task.internalError(event.getType()); } Preconditions.checkState(castEvent.getCausalEvent() != null); @@ -1418,7 +1418,7 @@ public void transition(TaskImpl task, TaskEvent event) { task.addDiagnosticInfo(terminateEvent.getDiagnosticInfo()); if (terminateEvent.isFromRecovery()) { if (LOG.isDebugEnabled()) { - LOG.debug("Recovered to KILLED, taskId=" + task.getTaskId()); + LOG.debug("Recovered to KILLED, taskId=" + task.getTaskID()); } } else { task.logJobHistoryTaskFailedEvent(TaskState.KILLED); @@ -1453,20 +1453,20 @@ public void onStateChanged(TaskImpl task, TaskStateInternal taskStateInternal) { // With this, recovery will end up failing for DAGs making use of InputInitializerEvents int succesfulAttemptInt = -1; if (successfulAttempt != null) { - succesfulAttemptInt = successfulAttempt.getID().getId(); + succesfulAttemptInt = successfulAttempt.getTaskAttemptID().getId(); } - task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskId(), + task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskID(), succesfulAttemptInt); } } private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg, TaskAttemptTerminationCause errorCause) { - if (commitAttempt != null && commitAttempt.equals(attempt.getID())) { + if (commitAttempt != null && commitAttempt.equals(attempt.getTaskAttemptID())) { LOG.info("Unsetting commit attempt: " + commitAttempt + " since attempt is being killed"); commitAttempt = null; } if (attempt != null && !attempt.isFinished()) { - eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getID(), logMsg, errorCause)); + eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getTaskAttemptID(), logMsg, errorCause)); } } @@ -1485,7 +1485,7 @@ public long getFirstAttemptStartTime() { readLock.lock(); try { // The first attempt will always have an index of 0. - return getAttempt(TezTaskAttemptID.getInstance(getTaskId(), 0)).getScheduleTime(); + return getAttempt(TezTaskAttemptID.getInstance(getTaskID(), 0)).getScheduleTime(); } finally { readLock.unlock(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java index a994359354..be4ee6068e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java @@ -64,7 +64,7 @@ public TezRootInputInitializerContextImpl( @Override public ApplicationId getApplicationId() { - return vertex.getVertexId().getDAGId().getApplicationId(); + return vertex.getVertexId().getDAGID().getApplicationId(); } @Override 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 f22f6de250..934dd4e76e 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 @@ -1590,7 +1590,7 @@ private void computeProgress() { if (LOG.isDebugEnabled()) { if (!ProgressHelper.isProgressWithinRange(taskProg)) { LOG.debug("progress update: vertex={}, task={} incorrect; range={}", - getName(), task.getTaskId(), taskProg); + getName(), task.getTaskID(), taskProg); } } accProg += ProgressHelper.processProgress(taskProg); @@ -2052,7 +2052,7 @@ public void doneReconfiguringVertex() { */ public void handle(VertexEvent event) { if (LOG.isDebugEnabled()) { - LOG.debug("Processing VertexEvent " + event.getVertexId() + LOG.debug("Processing VertexEvent " + event.getVertexID() + " of type " + event.getType() + " while in state " + getInternalState() + ". Event: " + event); } @@ -2114,7 +2114,7 @@ protected void addTask(Task task) { lazyTasksCopyNeeded = false; } } - tasks.put(task.getTaskId(), task); + tasks.put(task.getTaskID(), task); // TODO Metrics //metrics.waitingTask(task); } @@ -2410,7 +2410,7 @@ void tryEnactKill(VertexTerminationCause trigger, LOG.info(msg); for (Task task : tasks.values()) { eventHandler.handle( // attempt was terminated because the vertex is shutting down - new TaskEventTermination(task.getTaskId(), errCause, msg)); + new TaskEventTermination(task.getTaskID(), errCause, msg)); } } } @@ -2649,7 +2649,7 @@ private void createTasks() { this.addTask(task); if(LOG.isDebugEnabled()) { LOG.debug("Created task for vertex " + logIdentifier + ": " + - task.getTaskId()); + task.getTaskID()); } } } @@ -2663,7 +2663,7 @@ private void addTasks(int newNumTasks) { this.numTasks++; if(LOG.isDebugEnabled()) { LOG.debug("Created task for vertex " + logIdentifier + ": " + - task.getTaskId()); + task.getTaskID()); } } } @@ -2919,7 +2919,7 @@ private static List getTaskAttemptIdentifiers(DAG dag, List attempts = new ArrayList(taIds.size()); String dagName = dag.getName(); for (TezTaskAttemptID taId : taIds) { - String vertexName = dag.getVertex(taId.getTaskID().getVertexID()).getName(); + String vertexName = dag.getVertex(taId.getVertexID()).getName(); attempts.add(getTaskAttemptIdentifier(dagName, vertexName, taId)); } return attempts; @@ -3639,7 +3639,7 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { TezTaskAttemptID taId = completionEvent.getTaskAttemptId(); vertex.vertexManager.onSourceTaskCompleted( getTaskAttemptIdentifier(vertex.dag.getName(), - vertex.dag.getVertex(taId.getTaskID().getVertexID()).getName(), + vertex.dag.getVertex(taId.getVertexID()).getName(), taId)); } catch (AMUserCodeException e) { String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier(); @@ -3692,8 +3692,8 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { Task task = vertex.tasks.get(taskEvent.getTaskID()); if (taskEvent.getState() == TaskState.SUCCEEDED) { taskSucceeded(vertex, task); - if (!vertex.completedTasksStatsCache.containsTask(task.getTaskId())) { - vertex.completedTasksStatsCache.addTask(task.getTaskId()); + if (!vertex.completedTasksStatsCache.containsTask(task.getTaskID())) { + vertex.completedTasksStatsCache.addTask(task.getTaskID()); vertex.completedTasksStatsCache.mergeFrom(((TaskImpl) task).getStatistics()); } } else if (taskEvent.getState() == TaskState.FAILED) { @@ -3725,7 +3725,7 @@ private void taskSucceeded(VertexImpl vertex, Task task) { private void taskFailed(VertexImpl vertex, Task task) { vertex.failedTaskCount++; vertex.addDiagnostic("Task failed" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", diagnostics=" + task.getDiagnostics()); // TODO Metrics //vertex.metrics.failedTask(task); @@ -4090,7 +4090,7 @@ private void handleRoutedTezEvents(List tezEvents, boolean isPendingEv pendingTaskEvents.add(tezEvent); } else { // event not from this vertex. must have come from source vertex. - int srcTaskIndex = sourceMeta.getTaskAttemptID().getTaskID().getId(); + int srcTaskIndex = sourceMeta.getTaskID().getId(); Vertex edgeVertex = getDAG().getVertex(sourceMeta.getTaskVertexName()); Edge srcEdge = sourceVertices.get(edgeVertex); if (srcEdge == null) { @@ -4132,7 +4132,7 @@ private void handleRoutedTezEvents(List tezEvents, boolean isPendingEv Preconditions.checkArgument(target != null, "Event sent to unkown vertex: " + vmEvent.getTargetVertexName()); TezTaskAttemptID srcTaId = sourceMeta.getTaskAttemptID(); - if (srcTaId.getTaskID().getVertexID().equals(vertexId)) { + if (srcTaId.getVertexID().equals(vertexId)) { // this is the producer tasks' vertex vmEvent.setProducerAttemptIdentifier( getTaskAttemptIdentifier(dag.getName(), getName(), srcTaId)); 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 57c8c72be1..c3d49ea840 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 @@ -226,7 +226,7 @@ public synchronized void scheduleTasks(List tasks) { @Override public synchronized void scheduleVertexTasks(List tasks) { checkAndThrowIfDone(); - List schedTasks = new ArrayList(tasks.size()); + List schedTasks = new ArrayList<>(tasks.size()); for (TaskWithLocationHint task : tasks) { schedTasks.add(ScheduleTaskRequest.create( task.getTaskIndex(), task.getTaskLocationHint())); @@ -292,7 +292,7 @@ public void sendEventToProcessor(Collection events, int ta // attempt id won't be used anyway EventMetaData destinationMeta = new EventMetaData(EventProducerConsumerType.PROCESSOR, managedVertex.getName(), managedVertex.getName(), - TezTaskAttemptID.getInstance(managedVertex.getTask(taskId).getTaskId(), -1)); + TezTaskAttemptID.getInstance(managedVertex.getTask(taskId).getTaskID(), -1)); tezEvent.setDestinationInfo(destinationMeta); tezEvents.add(tezEvent); } @@ -571,7 +571,7 @@ private void sendInternalError(Exception e) { // state change must be triggered via an event transition LOG.error("Error after vertex manager callback " + managedVertex.getLogIdentifier(), e); appContext.getEventHandler().handle( - (new DAGEventInternalError(managedVertex.getVertexId().getDAGId(), + (new DAGEventInternalError(managedVertex.getVertexId().getDAGID(), "Error in VertexManager for vertex: " + managedVertex.getLogIdentifier() + ", error=" + ExceptionUtils.getStackTrace(e)))); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index 7aad8f642b..940bb23025 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -271,7 +271,7 @@ public void notifyAttemptStatusUpdate(TezTaskAttemptID taId, /** * Absorbs one TaskAttemptStatus * - * @param reportedStatus the status report that we got from a task attempt + * @param reportedState the status report that we got from a task attempt * that we want to fold into the speculation data for this job * @param timestamp the time this status corresponds to. This matters * because statuses contain progress. @@ -331,7 +331,7 @@ public void handle(SpeculatorEvent event) { // with private long speculationValue(Task task, long now, boolean shouldUseTimeout) { Map attempts = task.getAttempts(); - TezTaskID taskID = task.getTaskId(); + TezTaskID taskID = task.getTaskID(); long acceptableRuntime = Long.MIN_VALUE; long result = Long.MIN_VALUE; @@ -359,7 +359,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { if (++numberRunningAttempts > 1) { return ALREADY_SPECULATING; } - runningTaskAttemptID = taskAttempt.getID(); + runningTaskAttemptID = taskAttempt.getTaskAttemptID(); long taskAttemptStartTime = estimator.attemptEnrolledTime(runningTaskAttemptID); @@ -404,7 +404,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { .hasStagnatedProgress(runningTaskAttemptID, now)) { // Stats have stagnated for a while, simulate heart-beat. // Now simulate the heart-beat - statusUpdate(taskAttempt.getID(), taskAttempt.getState(), + statusUpdate(taskAttempt.getTaskAttemptID(), taskAttempt.getState(), clock.getTime()); } } else { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java index d2587b5190..22c5b26a7c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java @@ -53,8 +53,8 @@ public void run() { BaseHttpConnection httpConnection = null; try { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerTaskAttemptFailed( - nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGId(). - getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGId().getId(), + nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGID(). + getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGID().getId(), taskAttemptID.toString(), false); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "FailedTaskAttemptDelete", jobTokenSecretManager); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java index f7fee3acfe..107fbf62c7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java @@ -42,7 +42,7 @@ public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId, } public TezTaskAttemptID getAttemptID() { - return this.attempt.getID(); + return this.attempt.getTaskAttemptID(); } public TaskAttempt getAttempt() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 8e6bfe79a5..8a24ad3426 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -389,7 +389,7 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { String msg = "Error in TaskScheduler for handling Task De-allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + attempt.getID(); + + ", taskAttemptId=" + attempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -403,10 +403,10 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { ContainerId attemptContainerId = attempt.getAssignedContainerID(); if(!wasContainerAllocated) { - LOG.info("Task: " + attempt.getID() + + LOG.info("Task: " + attempt.getTaskAttemptID() + " has no container assignment in the scheduler"); if (attemptContainerId != null) { - LOG.error("No container allocated to task: " + attempt.getID() + LOG.error("No container allocated to task: " + attempt.getTaskAttemptID() + " according to scheduler. Task reported container id: " + attemptContainerId); } @@ -425,7 +425,7 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { Container container = amContainer.getContainer(); sendEvent(new AMNodeEventTaskAttemptEnded(container.getNodeId(), event.getSchedulerId(), attemptContainerId, - attempt.getID(), event.getState() == TaskAttemptState.FAILED)); + attempt.getTaskAttemptID(), event.getState() == TaskAttemptState.FAILED)); } } } @@ -458,7 +458,7 @@ private void handleTASucceeded(AMSchedulerEventTAEnded event) { String msg = "Error in TaskScheduler for handling Task De-allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + attempt.getID(); + + ", taskAttemptId=" + attempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -468,7 +468,7 @@ private void handleTASucceeded(AMSchedulerEventTAEnded event) { } if (!wasContainerAllocated) { - LOG.error("De-allocated successful task: " + attempt.getID() + LOG.error("De-allocated successful task: " + attempt.getTaskAttemptID() + ", but TaskScheduler reported no container assigned to task"); } } @@ -483,15 +483,15 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { if (taskAffinity != null) { Vertex vertex = appContext.getCurrentDAG().getVertex(taskAffinity.getVertexName()); Objects.requireNonNull(vertex, "Invalid vertex in task based affinity " + taskAffinity - + " for attempt: " + taskAttempt.getID()); + + " for attempt: " + taskAttempt.getTaskAttemptID()); int taskIndex = taskAffinity.getTaskIndex(); Preconditions.checkState(taskIndex >=0 && taskIndex < vertex.getTotalTasks(), "Invalid taskIndex in task based affinity " + taskAffinity - + " for attempt: " + taskAttempt.getID()); + + " for attempt: " + taskAttempt.getTaskAttemptID()); TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt(); if (affinityAttempt != null) { Objects.requireNonNull(affinityAttempt.getAssignedContainerID(), - affinityAttempt.getID() == null ? null : affinityAttempt.getID().toString()); + affinityAttempt.getTaskAttemptID() == null ? null : affinityAttempt.getTaskAttemptID().toString()); try { taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt, event.getCapability(), @@ -503,7 +503,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { String msg = "Error in TaskScheduler for handling Task Allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + taskAttempt.getID(); + + ", taskAttemptId=" + taskAttempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -513,7 +513,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { return; } LOG.info("No attempt for task affinity to " + taskAffinity + " for attempt " - + taskAttempt.getID() + " Ignoring."); + + taskAttempt.getTaskAttemptID() + " Ignoring."); // fall through with null hosts/racks } else { hosts = (locationHint.getHosts() != null) ? locationHint @@ -536,7 +536,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { String msg = "Error in TaskScheduler for handling Task Allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + taskAttempt.getID(); + + ", taskAttemptId=" + taskAttempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -552,7 +552,7 @@ private void handleTAStateUpdated(AMSchedulerEventTAStateUpdated event) { String msg = "Error in TaskScheduler for handling Task State Update" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + event.getTaskAttempt().getID() + + ", taskAttemptId=" + event.getTaskAttempt().getTaskAttemptID() + ", state=" + event.getState(); LOG.error(msg, e); sendEvent( @@ -763,7 +763,7 @@ public synchronized void taskAllocated(int schedulerId, Object task, event.getContainerContext(), event.getLauncherId(), event.getTaskCommId())); } } - sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getID(), + sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getTaskAttemptID(), event.getRemoteTaskSpec(), event.getContainerContext().getLocalResources(), event .getContainerContext().getCredentials(), event.getPriority())); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java index 92e5817243..c3aae35910 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java @@ -41,7 +41,7 @@ public AMContainerEventLaunchRequest(ContainerId containerId, } public TezDAGID getDAGId() { - return this.vertexId.getDAGId(); + return this.vertexId.getDAGID(); } public TezVertexID getVertexId() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java index 08d754d8a0..00cd26e2ce 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java @@ -760,7 +760,7 @@ public void getTasksInfo() { ArrayList> tasksInfo = new ArrayList>(); for(Task t : tasks) { Map taskInfo = new HashMap(); - taskInfo.put("id", t.getTaskId().toString()); + taskInfo.put("id", t.getTaskID().toString()); taskInfo.put("progress", Float.toString(t.getProgress())); taskInfo.put("status", t.getState().toString()); @@ -810,7 +810,7 @@ else if(!attemptIDs.isEmpty()) { } TaskAttempt attempt = task. - getAttempt(TezTaskAttemptID.getInstance(task.getTaskId(), indexes.get(2))); + getAttempt(TezTaskAttemptID.getInstance(task.getTaskID(), indexes.get(2))); if(attempt == null) { continue; } @@ -858,7 +858,7 @@ public void getAttemptsInfo() { ArrayList> attemptsInfo = new ArrayList>(); for(TaskAttempt a : attempts) { Map attemptInfo = new HashMap(); - attemptInfo.put("id", a.getID().toString()); + attemptInfo.put("id", a.getTaskAttemptID().toString()); attemptInfo.put("progress", Float.toString(a.getProgress())); attemptInfo.put("status", a.getState().toString()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java index dfa6bbdf91..240d8bd96d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java @@ -18,9 +18,10 @@ package org.apache.tez.dag.history; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; -public class DAGHistoryEvent { +public class DAGHistoryEvent implements DAGIDAware { private final HistoryEvent historyEvent; private final TezDAGID dagID; @@ -39,7 +40,8 @@ public HistoryEvent getHistoryEvent() { return historyEvent; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return this.dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java index 4fa1926ae9..f4dd789fd9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java @@ -124,7 +124,7 @@ public void serviceStop() throws Exception { * @throws IOException */ public void handleCriticalEvent(DAGHistoryEvent event) throws IOException { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); String dagIdStr = "N/A"; if(dagId != null) { dagIdStr = dagId.toString(); @@ -161,7 +161,7 @@ public void handleCriticalEvent(DAGHistoryEvent event) throws IOException { } private boolean shouldLogEvent(DAGHistoryEvent event) { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryLogLevel dagLogLevel = null; if (dagId != null) { @@ -207,7 +207,7 @@ private boolean shouldLogTaskAttemptEvents(DAGHistoryEvent event, HistoryLogLeve if (dagLogLevel == HistoryLogLevel.TASK_ATTEMPT && (eventType == HistoryEventType.TASK_ATTEMPT_STARTED || eventType == HistoryEventType.TASK_ATTEMPT_FINISHED)) { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); Set filters = null; if (dagId != null) { filters = dagIdToTaskAttemptFilters.get(dagId); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index 0a7ef561f7..6d1d6cd3ff 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -32,6 +32,7 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGFinishedProto; @@ -40,7 +41,7 @@ import com.google.common.primitives.Ints; import com.google.protobuf.ByteString; -public class DAGFinishedEvent implements HistoryEvent, SummaryEvent { +public class DAGFinishedEvent implements HistoryEvent, SummaryEvent, DAGIDAware { private TezDAGID dagID; private long startTime; @@ -179,7 +180,8 @@ public DAGState getState() { return state; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 9e6c8b282b..4df116ad37 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -25,12 +25,13 @@ import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGInitializedProto; -public class DAGInitializedEvent implements HistoryEvent { +public class DAGInitializedEvent implements HistoryEvent, DAGIDAware { private TezDAGID dagID; private long initTime; @@ -101,7 +102,8 @@ public long getInitTime() { return this.initTime; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index f1fdcac809..9e1f6c4a50 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -25,10 +25,11 @@ import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGStartedProto; -public class DAGStartedEvent implements HistoryEvent { +public class DAGStartedEvent implements HistoryEvent, DAGIDAware { private TezDAGID dagID; private long startTime; @@ -97,7 +98,8 @@ public long getStartTime() { return this.startTime; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index e04ee80188..bed0f6f25a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import org.apache.tez.dag.records.DAGIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -42,7 +43,7 @@ import org.apache.tez.dag.utils.ProtoUtils; -public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent { +public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent, DAGIDAware { private static final Logger LOG = LoggerFactory.getLogger(DAGSubmittedEvent.class); @@ -174,7 +175,8 @@ public DAGProtos.DAGPlan getDAGPlan() { return this.dagPlan; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index 18596495be..ce2eb729f6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -27,6 +27,7 @@ import org.apache.tez.common.TezConverterUtils; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +51,7 @@ import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto; import org.apache.tez.runtime.api.impl.TezEvent; -public class TaskAttemptFinishedEvent implements HistoryEvent { +public class TaskAttemptFinishedEvent implements HistoryEvent, TaskAttemptIDAware { private static final Logger LOG = LoggerFactory.getLogger(TaskAttemptFinishedEvent.class); @@ -307,6 +308,7 @@ public String toString() { return sb.toString(); } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index a49e47c97f..10ba439e1c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -27,10 +27,11 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskAttemptStartedProto; -public class TaskAttemptStartedEvent implements HistoryEvent { +public class TaskAttemptStartedEvent implements HistoryEvent, TaskAttemptIDAware { private TezTaskAttemptID taskAttemptId; private String inProgressLogsUrl; @@ -113,6 +114,7 @@ public String toString() { + ", nodeId=" + nodeId; } + @Override public TezTaskAttemptID getTaskAttemptID() { return this.taskAttemptId; } 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 dd7afdfc7c..215968274c 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 @@ -24,6 +24,7 @@ import com.google.protobuf.CodedOutputStream; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.dag.records.TaskIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -34,7 +35,7 @@ import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskFinishedProto; -public class TaskFinishedEvent implements HistoryEvent { +public class TaskFinishedEvent implements HistoryEvent, TaskIDAware { private static final Logger LOG = LoggerFactory.getLogger(TaskFinishedEvent.class); @@ -160,6 +161,7 @@ public String toString() { return sb.toString(); } + @Override public TezTaskID getTaskID() { return taskID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index cc629699a4..577aef963c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -25,10 +25,11 @@ import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskStartedProto; -public class TaskStartedEvent implements HistoryEvent { +public class TaskStartedEvent implements HistoryEvent, TaskIDAware { private TezTaskID taskID; private String vertexName; @@ -104,6 +105,7 @@ public String toString() { + ", launchTime=" + startTime; } + @Override public TezTaskID getTaskID() { return taskID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index 8ff86b8af1..ca2cafc834 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -99,7 +99,7 @@ public TezVertexID getVertexID() { @Override public void toSummaryProtoStream(OutputStream outputStream) throws IOException { SummaryEventProto.Builder builder = RecoveryProtos.SummaryEventProto.newBuilder() - .setDagId(vertexID.getDAGId().toString()) + .setDagId(vertexID.getDAGID().toString()) .setTimestamp(commitStartTime) .setEventType(getEventType().ordinal()) .setEventPayload( diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index a2e20397cf..2f4fac079f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -30,6 +30,7 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos.EdgeManagerDescriptorProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.RootInputSpecUpdateProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexConfigurationDoneProto; @@ -37,7 +38,7 @@ import com.google.common.collect.Maps; -public class VertexConfigurationDoneEvent implements HistoryEvent { +public class VertexConfigurationDoneEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private long reconfigureDoneTime; @@ -182,6 +183,7 @@ public String toString() { + ", setParallelismCalledFlag=" + setParallelismCalledFlag; } + @Override public TezVertexID getVertexID() { return this.vertexID; } 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 58cb628b28..24612a6a2e 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,6 +25,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; +import org.apache.tez.dag.records.VertexIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -39,7 +40,7 @@ import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishStateProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishedProto; -public class VertexFinishedEvent implements HistoryEvent, SummaryEvent { +public class VertexFinishedEvent implements HistoryEvent, SummaryEvent, VertexIDAware { private static final Logger LOG = LoggerFactory.getLogger(VertexFinishedEvent.class); @@ -157,6 +158,7 @@ public String toString() { + (servicePluginInfo != null ? servicePluginInfo : "null"); } + @Override public TezVertexID getVertexID() { return this.vertexID; } @@ -207,7 +209,7 @@ public void toSummaryProtoStream(OutputStream outputStream) throws IOException { .build(); SummaryEventProto.Builder builder = RecoveryProtos.SummaryEventProto.newBuilder() - .setDagId(vertexID.getDAGId().toString()) + .setDagId(vertexID.getDAGID().toString()) .setTimestamp(finishTime) .setEventType(getEventType().ordinal()) .setEventPayload(finishStateProto.toByteString()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index e7452e6770..a019fe3784 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -35,6 +35,7 @@ import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.utils.TezEventUtils; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexInitializedProto; @@ -42,7 +43,7 @@ import com.google.common.collect.Lists; -public class VertexInitializedEvent implements HistoryEvent { +public class VertexInitializedEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private String vertexName; @@ -180,8 +181,9 @@ public String toString() { + (servicePluginInfo != null ? servicePluginInfo : "null"); } + @Override public TezVertexID getVertexID() { - return this.vertexID; + return vertexID; } public long getInitRequestedTime() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index 4a3e05f1f4..cdd73b4483 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -26,9 +26,10 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexStartedProto; -public class VertexStartedEvent implements HistoryEvent { +public class VertexStartedEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private long startRequestedTime; @@ -94,8 +95,9 @@ public String toString() { + ", startedTime=" + startTime; } + @Override public TezVertexID getVertexID() { - return this.vertexID; + return vertexID; } public long getStartRequestedTime() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java index e60575f38e..8f4cd1fa6a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java @@ -341,7 +341,7 @@ private static JSONObject convertContainerStoppedEvent(ContainerStoppedEvent eve private static JSONObject convertDAGFinishedEvent(DAGFinishedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -384,7 +384,7 @@ private static JSONObject convertDAGFinishedEvent(DAGFinishedEvent event) throws private static JSONObject convertDAGInitializedEvent(DAGInitializedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -416,7 +416,7 @@ private static JSONObject convertDAGInitializedEvent(DAGInitializedEvent event) private static JSONObject convertDAGStartedEvent(DAGStartedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -439,7 +439,7 @@ private static JSONObject convertDAGStartedEvent(DAGStartedEvent event) throws J private static JSONObject convertDAGSubmittedEvent(DAGSubmittedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -602,7 +602,7 @@ private static JSONObject convertTaskAttemptStartedEvent(TaskAttemptStartedEvent containerEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.CONTAINER_ID); JSONObject taskEntity = new JSONObject(); - taskEntity.put(ATSConstants.ENTITY, event.getTaskAttemptID().getTaskID().toString()); + taskEntity.put(ATSConstants.ENTITY, event.getTaskID().toString()); taskEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name()); relatedEntities.put(nodeEntity); @@ -667,7 +667,7 @@ private static JSONObject convertTaskStartedEvent(TaskStartedEvent event) throws // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getTaskID().getVertexID().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); @@ -775,7 +775,7 @@ private static JSONObject convertVertexInitializedEvent(VertexInitializedEvent e // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().getDAGId().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getDAGID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); @@ -815,7 +815,7 @@ private static JSONObject convertVertexStartedEvent(VertexStartedEvent event) // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().getDAGId().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getDAGID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index 45e7d2fc9c..5b5a9c7261 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -300,7 +300,7 @@ public void handle(DAGHistoryEvent event) throws IOException { return; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = (DAGSubmittedEvent) event.getHistoryEvent(); @@ -337,8 +337,8 @@ public void handle(DAGHistoryEvent event) throws IOException { if (summaryEvent.writeToRecoveryImmediately()) { handleRecoveryEvent(event); // outputStream may already be closed and removed - if (outputStreamMap.containsKey(event.getDagID())) { - doFlush(outputStreamMap.get(event.getDagID()), + if (outputStreamMap.containsKey(event.getDAGID())) { + doFlush(outputStreamMap.get(event.getDAGID()), appContext.getClock().getTime()); } } else { @@ -350,7 +350,7 @@ public void handle(DAGHistoryEvent event) throws IOException { } if (eventType.equals(HistoryEventType.DAG_FINISHED)) { LOG.info("DAG completed" - + ", dagId=" + event.getDagID() + + ", dagId=" + event.getDAGID() + ", queueSize=" + eventQueue.size()); completedDAGs.add(dagId); if (outputStreamMap.containsKey(dagId)) { @@ -359,7 +359,7 @@ public void handle(DAGHistoryEvent event) throws IOException { outputStreamMap.remove(dagId); } catch (IOException ioe) { LOG.warn("Error when trying to flush/close recovery file for" - + " dag, dagId=" + event.getDagID()); + + " dag, dagId=" + event.getDAGID()); } } } @@ -436,7 +436,7 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { LOG.debug("Handling recovery event of type " + event.getHistoryEvent().getEventType()); } - TezDAGID dagID = event.getDagID(); + TezDAGID dagID = event.getDAGID(); if (completedDAGs.contains(dagID)) { // no need to recover completed DAGs diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index aa5d3b9535..6c0231c9b0 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -261,7 +261,7 @@ public void testBasicEvents() throws Exception { Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); VertexImpl vImpl = (VertexImpl) dagImpl.getVertex(vB.getName()); TaskImpl tImpl = (TaskImpl) vImpl.getTask(1); - TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); List tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(2, tEvents.size()); // 2 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -275,7 +275,7 @@ public void testBasicEvents() throws Exception { (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0)); vImpl = (VertexImpl) dagImpl.getVertex(vC.getName()); tImpl = (TaskImpl) vImpl.getTask(1); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(2, tEvents.size()); // 2 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -289,7 +289,7 @@ public void testBasicEvents() throws Exception { (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0)); vImpl = (VertexImpl) dagImpl.getVertex(vD.getName()); tImpl = (TaskImpl) vImpl.getTask(1); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(1, tEvents.size()); // 1 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -398,17 +398,17 @@ public void testMixedEdgeRouting() throws Exception { // vC uses on demand routing and its task does not provide events VertexImpl vImpl = (VertexImpl) dagImpl.getVertex(vC.getName()); TaskImpl tImpl = (TaskImpl) vImpl.getTask(0); - TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(0, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); // vD is mixed mode and only 1 out of 2 edges does legacy routing with task providing events vImpl = (VertexImpl) dagImpl.getVertex(vD.getName()); tImpl = (TaskImpl) vImpl.getTask(0); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(1, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); // vE has single legacy edge and does not use on demand routing and its task provides events vImpl = (VertexImpl) dagImpl.getVertex(vE.getName()); tImpl = (TaskImpl) vImpl.getTask(0); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(1, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); tezClient.stop(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index b6f50301c7..302281af9f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -326,7 +326,7 @@ public void testSingleTaskSpeculation() throws Exception { Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(entry.getValue().intValue(), task.getAttempts().size()); if (entry.getValue() > 1) { - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, @@ -369,7 +369,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(ASSERT_SPECULATIONS_COUNT_MSG, 2, task.getAttempts().size()); - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, @@ -380,7 +380,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { .getValue()); Assert.assertEquals(1, dagImpl.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); - org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getTaskID().getVertexID()); + org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getVertexID()); Assert.assertEquals(1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); } @@ -508,7 +508,7 @@ public void testBasicSpeculationNotUseful() throws Exception { Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(2, task.getAttempts().size()); - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed speculative attempt as"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_INEFFECTIVE_SPECULATION, @@ -517,7 +517,7 @@ public void testBasicSpeculationNotUseful() throws Exception { .getValue()); Assert.assertEquals(1, dagImpl.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); - org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getTaskID().getVertexID()); + org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getVertexID()); Assert.assertEquals(1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); tezClient.stop(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java index 7611f1c44e..436c522ef7 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java @@ -167,7 +167,7 @@ private class VertexEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - Vertex vertex = dag.getVertex(event.getVertexId()); + Vertex vertex = dag.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -176,7 +176,7 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - Vertex vertex = dag.getVertex(event.getTaskID().getVertexID()); + Vertex vertex = dag.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); ((EventHandler) task).handle(event); } @@ -563,7 +563,7 @@ public void testVertexCommit_OnDAGSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertNull(v1.getTerminationCause()); @@ -594,7 +594,7 @@ public void testVertexCommit_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -633,7 +633,7 @@ public void testVertexCommitFail1_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -669,7 +669,7 @@ public void testVertexCommitFail2_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -707,7 +707,7 @@ public void testVertexKilledWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // kill dag which will trigger the vertex killed event @@ -746,7 +746,7 @@ public void testVertexRescheduleWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // reschedule task @@ -786,7 +786,7 @@ public void testVertexRouteEventErrorWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // reschedule task @@ -831,7 +831,7 @@ public void testVertexInternalErrorWhileCommiting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // internal error @@ -875,11 +875,11 @@ public void testDAGCommitSucceeded_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); CountingOutputCommitter v12OutputCommitter = (CountingOutputCommitter) v1 @@ -931,11 +931,11 @@ public void testDAGCommitFail1_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); @@ -992,11 +992,11 @@ public void testDAGCommitFail2_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); @@ -1052,11 +1052,11 @@ public void testDAGCommitSucceeded1_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1110,11 +1110,11 @@ public void testDAGCommitSucceeded2_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1170,11 +1170,11 @@ public void testDAGCommitSucceeded3_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1235,11 +1235,11 @@ public void testDAGCommitFail1_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1294,11 +1294,11 @@ public void testDAGCommitFail2_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1353,11 +1353,11 @@ public void testDAGCommitFail3_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1416,11 +1416,11 @@ public void testDAGCommitFail4_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1475,11 +1475,11 @@ public void testDAGInternalErrorWhileCommiting_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEvent(dag.getID(), DAGEventType.INTERNAL_ERROR)); @@ -1537,11 +1537,11 @@ private void _testDAGTerminatedWhileCommitting1_OnDAGSuccess(DAGTerminationCause VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEventTerminateDag(dag.getID(), terminationCause, null)); @@ -1602,11 +1602,11 @@ private void _testDAGTerminatedWhileCommitting1_OnVertexSuccess(DAGTerminationCa VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v3.getState()); // dag is still in RUNNING because v3 has not completed @@ -1672,11 +1672,11 @@ private void _testDAGKilledWhileRunning_OnVertexSuccess(DAGTerminationCause term VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v3.getState()); // dag is still in RUNNING because v3 has not completed @@ -1731,11 +1731,11 @@ public void testDAGCommitVertexRerunWhileCommitting_OnDAGSuccess() throws Except VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); TezTaskID newTaskId = TezTaskID.getInstance(v1.getVertexId(), 1); @@ -1790,11 +1790,11 @@ public void testDAGCommitInternalErrorWhileCommiting_OnDAGSuccess() throws Excep VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEvent(dag.getID(), DAGEventType.INTERNAL_ERROR)); @@ -1840,11 +1840,11 @@ public void testVertexGroupCommitFinishedEventFail_OnVertexSuccess() throws Exce VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); CountingOutputCommitter v12OutputCommitter = (CountingOutputCommitter) v1 .getOutputCommitter("v12Out"); @@ -1895,11 +1895,11 @@ public void testDAGCommitStartedEventFail_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.FAILED); Assert.assertEquals(DAGTerminationCause.RECOVERY_FAILURE, dag.getTerminationCause()); @@ -1959,11 +1959,11 @@ private void _testCommitCanceled_OnDAGSuccess(DAGTerminationCause terminationCau VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); // mean the commits have been submitted to ThreadPool @@ -2110,7 +2110,7 @@ public void verifyDAGFinishedEvent(TezDAGID dagId, int expectedTimes) { for (HistoryEvent event : historyEvents) { if (event.getEventType() == HistoryEventType.DAG_FINISHED) { DAGFinishedEvent startedEvent = (DAGFinishedEvent)event; - if (startedEvent.getDagID().equals(dagId)) { + if (startedEvent.getDAGID().equals(dagId)) { actualTimes ++; } } 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 2f2b3b8b8f..f2b16a9094 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 @@ -221,7 +221,7 @@ private DAGImpl chooseDAG(TezDAGID curDAGId) { private class DagEventDispatcher implements EventHandler { @Override public void handle(DAGEvent event) { - DAGImpl dag = chooseDAG(event.getDAGId()); + DAGImpl dag = chooseDAG(event.getDAGID()); dag.handle(event); } } @@ -230,9 +230,9 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - TezDAGID id = event.getTaskID().getVertexID().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getTaskID().getVertexID()); + Vertex vertex = handler.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); ((EventHandler)task).handle(event); } @@ -249,10 +249,10 @@ public void handle(TaskAttemptEvent event) { private class TaskAttemptEventDisptacher2 implements EventHandler { @Override public void handle(TaskAttemptEvent event) { - TezDAGID id = event.getTaskAttemptID().getTaskID().getVertexID().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getTaskAttemptID().getTaskID().getVertexID()); - Task task = vertex.getTask(event.getTaskAttemptID().getTaskID()); + Vertex vertex = handler.getVertex(event.getVertexID()); + Task task = vertex.getTask(event.getTaskID()); TaskAttempt ta = task.getAttempt(event.getTaskAttemptID()); ((EventHandler)ta).handle(event); } @@ -264,9 +264,9 @@ private class VertexEventDispatcher @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - TezDAGID id = event.getVertexId().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getVertexId()); + Vertex vertex = handler.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -1174,14 +1174,14 @@ public void testEdgeManager_RouteDataMovementEventToDestination() { VertexImpl v2 = (VertexImpl)dagWithCustomEdge.getVertex("vertex2"); dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); DataMovementEvent daEvent = DataMovementEvent.create(ByteBuffer.wrap(new byte[0])); TezEvent tezEvent = new TezEvent(daEvent, - new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); - v2.getTaskAttemptTezEvents(ta1.getID(), 0, 0, 1000); + v2.getTaskAttemptTezEvents(ta1.getTaskAttemptID(), 0, 0, 1000); dispatcher.await(); Assert.assertEquals(VertexState.FAILED, v2.getState()); @@ -1207,11 +1207,11 @@ public void testEdgeManager_RouteDataMovementEventToDestinationWithLegacyRouting dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); DataMovementEvent daEvent = DataMovementEvent.create(ByteBuffer.wrap(new byte[0])); TezEvent tezEvent = new TezEvent(daEvent, - new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle( new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); @@ -1239,13 +1239,13 @@ public void testEdgeManager_RouteInputSourceTaskFailedEventToDestinationLegacyRo dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputFailedEvent ifEvent = InputFailedEvent.create(0, 1); TezEvent tezEvent = new TezEvent(ifEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); - v2.getTaskAttemptTezEvents(ta1.getID(), 0, 0, 1000); + v2.getTaskAttemptTezEvents(ta1.getTaskAttemptID(), 0, 0, 1000); dispatcher.await(); Assert.assertEquals(VertexState.FAILED, v2.getState()); @@ -1270,11 +1270,11 @@ public void testEdgeManager_GetNumDestinationConsumerTasks() { dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputReadErrorEvent ireEvent = InputReadErrorEvent.create("", 0, 0); TezEvent tezEvent = new TezEvent(ireEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle( new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); @@ -1301,10 +1301,10 @@ public void testEdgeManager_RouteInputErrorEventToSource() { dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputReadErrorEvent ireEvent = InputReadErrorEvent.create("", 0, 0); TezEvent tezEvent = new TezEvent(ireEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); // 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 index 95ea8a030e..acbef67dc0 100644 --- 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 @@ -209,7 +209,7 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - TaskImpl task = (TaskImpl) dag.getVertex(event.getTaskID().getVertexID()) + TaskImpl task = (TaskImpl) dag.getVertex(event.getVertexID()) .getTask(event.getTaskID()); task.handle(event); } @@ -220,8 +220,7 @@ private class TaskAttemptEventDispatcher implements EventHandler { @Override public void handle(TaskAttemptEvent event) { - Vertex vertex = dag.getVertex(event.getTaskAttemptID().getTaskID() - .getVertexID()); + Vertex vertex = dag.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskAttemptID().getTaskID()); TaskAttempt ta = task.getAttempt(event.getTaskAttemptID()); ((EventHandler) ta).handle(event); @@ -233,7 +232,7 @@ private class VertexEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - VertexImpl vertex = (VertexImpl) dag.getVertex(event.getVertexId()); + VertexImpl vertex = (VertexImpl) dag.getVertex(event.getVertexID()); vertex.handle(event); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java index 07c361aded..e2df050799 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java @@ -39,7 +39,7 @@ public class TestDAGScheduler { - class MockEventHandler implements EventHandler { + static class MockEventHandler implements EventHandler { TaskAttemptEventSchedule event; List events = Lists.newLinkedList(); @Override @@ -114,17 +114,17 @@ public void testConcurrencyLimit() { // schedule beyond limit and it gets scheduled mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 0)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 0)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(1, mockEventHandler.events.size()); mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 1)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 1)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(2, mockEventHandler.events.size()); mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 2)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 2)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(3, mockEventHandler.events.size()); @@ -139,34 +139,34 @@ public void testConcurrencyLimit() { // schedule beyond limit and it gets buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered @@ -174,14 +174,14 @@ public void testConcurrencyLimit() { scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(completed++))); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(completed++))); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; @@ -191,14 +191,12 @@ public void testConcurrencyLimit() { mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order - scheduled++; - } @Test(timeout=5000) @@ -220,7 +218,6 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { mockEventHandler); List mockAttempts = Lists.newArrayList(); - int completed = 0; int requested = 0; int scheduled = 0; scheduler.addVertexConcurrencyLimit(vId0, 1); // effective @@ -228,24 +225,24 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { // schedule beyond limit and it gets buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered @@ -253,7 +250,7 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(1))); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered - Assert.assertEquals(mockAttempts.get(0).getID(), + Assert.assertEquals(mockAttempts.get(0).getTaskAttemptID(), mockEventHandler.events.get(0).getTaskAttemptID()); // matches order } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java index 63137c716a..3b1db4c6d1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java @@ -164,9 +164,9 @@ public void testSourceRequestDelayed() { for (Event raw : args.getAllValues()) { TaskAttemptEventSchedule event = (TaskAttemptEventSchedule) raw; if (count < vertices[2].getTotalTasks() - 3) { - assertEquals(2, event.getTaskAttemptID().getTaskID().getVertexID().getId()); + assertEquals(2, event.getVertexID().getId()); } else { - assertEquals(4, event.getTaskAttemptID().getTaskID().getVertexID().getId()); + assertEquals(4, event.getVertexID().getId()); } count++; } @@ -364,7 +364,7 @@ private TaskAttempt createTaskAttempt(TezVertexID vertexId, int taskIdInt, int a TaskAttempt taskAttempt = mock(TaskAttempt.class); TezTaskID taskId = TezTaskID.getInstance(vertexId, taskIdInt); TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, attemptIdInt); - doReturn(taskAttemptId).when(taskAttempt).getID(); + doReturn(taskAttemptId).when(taskAttempt).getTaskAttemptID(); doReturn(vertexId).when(taskAttempt).getVertexID(); return taskAttempt; } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java index 1143395204..c9c99f5630 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java @@ -232,9 +232,9 @@ private void verifyEvents(TezTaskAttemptID srcTAID, LinkedHashMap tasks) { for (Task task : tasks) { - TezTaskID taskID = task.getTaskId(); + TezTaskID taskID = task.getTaskID(); reset(task); - doReturn(taskID).when(task).getTaskId(); + doReturn(taskID).when(task).getTaskID(); } } @@ -243,7 +243,7 @@ private LinkedHashMap mockTasks(TezVertexID vertexID, int numTa for (int i = 0 ; i < numTasks ; i++) { Task task = mock(Task.class); TezTaskID taskID = TezTaskID.getInstance(vertexID, i); - doReturn(taskID).when(task).getTaskId(); + doReturn(taskID).when(task).getTaskID(); tasks.put(taskID, task); } return tasks; 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 7a2a05fb26..dc59b4615c 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 @@ -425,7 +425,7 @@ taListener, taskConf, new SystemClock(), taImpl.handle(new TaskAttemptEventKillRequest(taskAttemptID, null, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT)); assertEquals(TaskAttemptStateInternal.KILL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in KILL_IN_PROGRESS state", eventHandler.internalError); @@ -480,7 +480,7 @@ public void testContainerTerminationWhileRunning() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -585,7 +585,7 @@ public void testContainerTerminatedWhileRunning() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -674,7 +674,7 @@ public void testContainerTerminatedAfterSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -764,7 +764,7 @@ public void testLastDataEventRecording() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -849,7 +849,7 @@ public void testFailure() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -879,7 +879,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.APPLICATION_ERROR, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.FAIL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in FAIL_IN_PROGRESS state", eventHandler.internalError); @@ -956,7 +956,7 @@ public void testFailureFatalError() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -986,7 +986,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.APPLICATION_ERROR, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.FAIL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in FAIL_IN_PROGRESS state", eventHandler.internalError); @@ -1064,7 +1064,7 @@ public void testProgressTimeStampUpdate() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1080,7 +1080,7 @@ public void testProgressTimeStampUpdate() throws Exception { verify(eventHandler, atLeast(1)).handle(arg.capture()); if (arg.getValue() instanceof TaskAttemptEventAttemptFailed) { TaskAttemptEventAttemptFailed fEvent = (TaskAttemptEventAttemptFailed) arg.getValue(); - assertEquals(taImpl.getID(), fEvent.getTaskAttemptID()); + assertEquals(taImpl.getTaskAttemptID(), fEvent.getTaskAttemptID()); assertEquals(TaskAttemptTerminationCause.NO_PROGRESS, fEvent.getTerminationCause()); taImpl.handle(fEvent); fail("Should not fail since the timestamps do not differ by progress interval config"); @@ -1135,7 +1135,7 @@ public void testStatusUpdateWithNullCounters() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -1203,7 +1203,7 @@ public void testProgressAfterSubmit() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); mockClock.incrementTime(20L); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1262,7 +1262,7 @@ public void testNoProgressFail() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1295,7 +1295,7 @@ public void testNoProgressFail() throws Exception { // failed event sent to self verify(eventHandler, atLeast(1)).handle(arg.capture()); TaskAttemptEventAttemptFailed fEvent = (TaskAttemptEventAttemptFailed) arg.getValue(); - assertEquals(taImpl.getID(), fEvent.getTaskAttemptID()); + assertEquals(taImpl.getTaskAttemptID(), fEvent.getTaskAttemptID()); assertEquals(TaskAttemptTerminationCause.NO_PROGRESS, fEvent.getTerminationCause()); assertEquals(TaskFailureType.NON_FATAL, fEvent.getTaskFailureType()); taImpl.handle(fEvent); @@ -1377,7 +1377,7 @@ public void testCompletedAtSubmitted() throws ServicePluginException { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1460,7 +1460,7 @@ public void testSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1548,7 +1548,7 @@ public void testContainerPreemptedAfterSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1640,7 +1640,7 @@ public void testNodeFailedNonLeafVertex() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1699,7 +1699,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.NODE_FAILED, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in KILLED state", eventHandler.internalError); @@ -1747,7 +1747,7 @@ public void testNodeFailedLeafVertex() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), true); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1847,7 +1847,7 @@ public void testMultipleOutputFailed() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -1876,6 +1876,7 @@ taListener, taskConf, new SystemClock(), TezTaskID destTaskID = mock(TezTaskID.class); TezVertexID destVertexID = mock(TezVertexID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); Vertex destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(11); @@ -1901,14 +1902,14 @@ taListener, taskConf, new SystemClock(), destTaskID = mock(TezTaskID.class); destVertexID = mock(TezVertexID.class); when(mockDestId2.getTaskID()).thenReturn(destTaskID); + when(mockDestId2.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(11); when(mockDAG.getVertex(destVertexID)).thenReturn(destVertex); taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); - assertEquals("Task attempt is not in FAILED state", taImpl.getState(), - TaskAttemptState.FAILED); + assertEquals("Task attempt is not in FAILED state", TaskAttemptState.FAILED, taImpl.getState()); assertEquals(TaskAttemptTerminationCause.OUTPUT_LOST, taImpl.getTerminationCause()); // verify unregister is not invoked again verify(mockHeartbeatHandler, times(1)).unregister(taskAttemptID); @@ -1949,7 +1950,7 @@ taListener, taskConf, new SystemClock(), taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID2 = taImpl2.getID(); + TezTaskAttemptID taskAttemptID2 = taImpl2.getTaskAttemptID(); taImpl2.handle(new TaskAttemptEventSchedule(taskAttemptID2, 0, 0)); taImpl2.handle(new TaskAttemptEventSubmitted(taskAttemptID2, contId)); @@ -1964,6 +1965,7 @@ taListener, taskConf, new SystemClock(), mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); //This should fail even when MAX_ALLOWED_OUTPUT_FAILURES_FRACTION is within limits, as @@ -1991,7 +1993,7 @@ taListener, taskConf, new SystemClock(), taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID3 = taImpl3.getID(); + TezTaskAttemptID taskAttemptID3 = taImpl3.getTaskAttemptID(); taImpl3.handle(new TaskAttemptEventSchedule(taskAttemptID3, 0, 0)); taImpl3.handle(new TaskAttemptEventSubmitted(taskAttemptID3, contId)); @@ -2006,6 +2008,7 @@ taListener, taskConf, new SystemClock(), mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); when(mockClock.getTime()).thenReturn(1000L); @@ -2073,7 +2076,7 @@ public void testTAFailureBasedOnRunningTasks() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -2102,6 +2105,7 @@ taListener, taskConf, new SystemClock(), TezTaskID destTaskID = mock(TezTaskID.class); TezVertexID destVertexID = mock(TezVertexID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); Vertex destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(5); @@ -2111,8 +2115,7 @@ taListener, taskConf, new SystemClock(), taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); // failure threshold is met due to running tasks. state is FAILED - assertEquals("Task attempt is not in FAILED state", taImpl.getState(), - TaskAttemptState.FAILED); + assertEquals("Task attempt is not in FAILED state", TaskAttemptState.FAILED, taImpl.getState()); } @SuppressWarnings("deprecation") @@ -2157,7 +2160,7 @@ taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), true); Assert.assertEquals(TaskAttemptStateInternal.NEW, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventKillRequest(taImpl.getID(), "kill it", + taImpl.handle(new TaskAttemptEventKillRequest(taImpl.getTaskAttemptID(), "kill it", TaskAttemptTerminationCause.TERMINATED_BY_CLIENT)); Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState()); @@ -2190,7 +2193,7 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource, null); TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); - when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); + when(destTaskAttemptId.getVertexID()).thenReturn(mock(TezVertexID.class)); when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))) .thenReturn(mock(Vertex.class)); @@ -2203,7 +2206,7 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, // the event is propagated to map task's event handler TaskAttemptEventOutputFailed outputFailedEvent = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent, 11); Assert.assertEquals(TaskAttemptStateInternal.NEW, sourceAttempt.getInternalState()); TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition() @@ -2236,13 +2239,13 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); // mapper task succeeded earlier - sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getID(), TaskAttemptEventType.TA_DONE)); + sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); Assert.assertEquals(TaskAttemptStateInternal.SUCCEEDED, sourceAttempt.getInternalState()); // the event is propagated to map task's event handler TezEvent tezEvent = new TezEvent(inputReadErrorEvent1, mockMeta); TaskAttemptEventOutputFailed outputFailedEvent = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent, 11); TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent); // SUCCEEDED, as we haven't reached the host limit fraction @@ -2252,7 +2255,7 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { // the second event is propagated to map task's event handler TezEvent tezEvent2 = new TezEvent(inputReadErrorEvent2, mockMeta); TaskAttemptEventOutputFailed outputFailedEvent2 = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent2, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent2, 11); TaskAttemptStateInternal resultState2 = new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent2); 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 a28e786c6e..f4e40f7333 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 @@ -386,9 +386,9 @@ public void testKillRunningTask() { LOG.info("--- START: testKillRunningTask ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); } @@ -398,20 +398,20 @@ public void testTooManyFailedAttempts() { LOG.info("--- START: testTooManyFailedAttempts ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID(), false); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), false); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -431,11 +431,11 @@ public void testTooManyAttempts() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); // attempt_0 - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING, 1); // attempt_1 + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING, 1); // attempt_1 - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.FAILED, 2); // attempt_2 -> reached 3 + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.FAILED, 2); // attempt_2 -> reached 3 assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -446,9 +446,9 @@ public void testFailedAttemptWithFatalError() { LOG.info("--- START: testFailedAttemptWithFatalError ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); mockTask.handle( - createTaskTAFailedEvent(mockTask.getLastAttempt().getID(), TaskFailureType.FATAL, null)); + createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID(), TaskFailureType.FATAL, null)); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); assertEquals(1, mockTask.failedAttempts); @@ -460,11 +460,11 @@ public void testKillRunningTaskPreviousKilledAttempts() { LOG.info("--- START: testKillRunningTaskPreviousKilledAttempts ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING); assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -478,9 +478,9 @@ public void testKillRunningTaskButAttemptSucceeds() { LOG.info("--- START: testKillRunningTaskButAttemptSucceeds ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -492,9 +492,9 @@ public void testKillRunningTaskButAttemptFails() { LOG.info("--- START: testKillRunningTaskButAttemptFails ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -507,8 +507,8 @@ public void testKillScheduledTaskAttempt() { LOG.info("--- START: testKillScheduledTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); - killScheduledTaskAttempt(mockTask.getLastAttempt().getID()); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); + killScheduledTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); // last killed attempt should be causal TA of next attempt Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } @@ -522,7 +522,7 @@ public void testLaunchTaskAttempt() { LOG.info("--- START: testLaunchTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); } @Test(timeout = 5000) @@ -534,9 +534,9 @@ public void testKillRunningTaskAttempt() { LOG.info("--- START: testKillRunningTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING); // last killed attempt should be causal TA of next attempt Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } @@ -550,9 +550,9 @@ public void testKillTaskAttemptServiceBusy() { LOG.info("--- START: testKillTaskAttemptServiceBusy ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); mockTask.handle(createTaskTAKilledEvent( - mockTask.getLastAttempt().getID(), new ServiceBusyEvent())); + mockTask.getLastAttempt().getTaskAttemptID(), new ServiceBusyEvent())); assertTaskRunningState(); verify(mockTask.getVertex(), times(0)).incrementKilledTaskAttemptCount(); verify(mockTask.getVertex(), times(1)).incrementRejectedTaskAttemptCount(); @@ -566,15 +566,15 @@ public void testKilledAttemptAtTaskKilled() { LOG.info("--- START: testKilledAttemptAtTaskKilled ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); // Send duplicate kill for same attempt // This will not happen in practice but this is to simulate handling // of killed attempts in killed state. - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -588,12 +588,12 @@ public void testKilledAttemptAtTaskFailed() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); for (int i = 0; i < mockTask.maxFailedAttempts; ++i) { - mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID())); } assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); // Send kill for an attempt - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); } @@ -609,7 +609,7 @@ public void testFetchedEventsModifyUnderlyingList() { scheduleTaskAttempt(taskId); sendTezEventsToTask(taskId, 2); TezTaskAttemptID attemptID = mockTask.getAttemptList().iterator().next() - .getID(); + .getTaskAttemptID(); fetchedList = mockTask.getTaskAttemptTezEvents(attemptID, 0, 100); assertEquals(2, fetchedList.size()); @@ -631,7 +631,7 @@ public void testTaskProgress() { scheduleTaskAttempt(taskId); float progress = 0f; assert (mockTask.getProgress() == progress); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); // update attempt1 progress = 50f; @@ -649,13 +649,13 @@ public void testTaskProgress() { // kill first attempt // should trigger a new attempt // as no successful attempts - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assert (mockTask.getAttemptList().size() == 2); assertEquals(1, mockTask.failedAttempts); verify(mockTask.getVertex(), times(1)).incrementFailedTaskAttemptCount(); assert (mockTask.getProgress() == 0f); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); progress = 50f; updateAttemptProgress(mockTask.getLastAttempt(), progress); assert (mockTask.getProgress() == progress); @@ -665,17 +665,17 @@ public void testTaskProgress() { public void testFailureDuringTaskAttemptCommit() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("First attempt should commit", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); // During the task attempt commit there is an exception which causes // the attempt to fail - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.FAILED); assertEquals(1, mockTask.getAttemptList().size()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assertEquals(2, mockTask.getAttemptList().size()); assertEquals(1, mockTask.failedAttempts); @@ -683,13 +683,13 @@ public void testFailureDuringTaskAttemptCommit() { Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); assertFalse("First attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("Second attempt should commit", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertTaskSucceededState(); } @@ -703,14 +703,14 @@ public void testEventBacklogDuringTaskAttemptCommit() { // simulate // task in scheduled state due to event backlog - real task done and calling canCommit assertFalse("Commit should return false to make running task wait", - mockTask.canCommit(mockTask.getLastAttempt().getID())); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("Task state in AM is running now. Can commit.", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertTaskSucceededState(); } @@ -720,13 +720,13 @@ public void testEventBacklogDuringTaskAttemptCommit() { public void testChangeCommitTaskAttempt() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); // Add a speculative task attempt that succeeds - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); @@ -735,24 +735,24 @@ public void testChangeCommitTaskAttempt() { Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); assertTrue("Second attempt should commit", - mockTask.canCommit(mockTask.getAttemptList().get(1).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(1).getTaskAttemptID())); assertFalse("First attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); // During the task attempt commit there is an exception which causes // the second attempt to fail updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.FAILED); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assertEquals(2, mockTask.getAttemptList().size()); assertFalse("Second attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(1).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(1).getTaskAttemptID())); assertTrue("First attempt should commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); updateAttemptState(mockTask.getAttemptList().get(0), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getAttemptList().get(0).getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getAttemptList().get(0).getTaskAttemptID())); assertTaskSucceededState(); } @@ -762,15 +762,15 @@ public void testChangeCommitTaskAttempt() { public void testTaskSucceedAndRetroActiveFailure() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should now have succeeded assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(mockTask.getLastAttempt().getID().getId())); + eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class); verify(mockHistoryHandler).handle(argumentCaptor.capture()); @@ -789,7 +789,7 @@ public void testTaskSucceedAndRetroActiveFailure() { TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, mockTezEvent, 1); mockTask.handle( - createTaskTAFailedEvent(mockTask.getLastAttempt().getID(), TaskFailureType.NON_FATAL, + createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent)); // The task should still be in the scheduled state @@ -811,19 +811,19 @@ public void testTaskSucceedAndRetroActiveFailure() { public void testTaskSucceedAndRetroActiveKilled() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should now have succeeded assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(mockTask.getLastAttempt().getID().getId())); + eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); eventHandler.events.clear(); // Now kill the attempt after it has succeeded - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should still be in the scheduled state assertTaskScheduledState(); @@ -862,25 +862,25 @@ public void testFailedThenSpeculativeFailed() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Fail the first attempt updateAttemptState(firstAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); // Now fail the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(specAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); } @@ -897,25 +897,25 @@ public void testFailedThenSpeculativeSucceeded() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Fail the first attempt updateAttemptState(firstAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); // Now succeed the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(specAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); } @@ -932,10 +932,10 @@ public void testKilledBeforeSpeculatedSucceeded() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); - mockTask.handle(createTaskTAKilledEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); // We need to manually override the current node id @@ -945,16 +945,16 @@ public void testKilledBeforeSpeculatedSucceeded() { mockNodeId = null; // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); mockNodeId = nodeId; MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(3, mockTask.getAttemptList().size()); // Now succeed the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(specAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.SUCCEEDED, mockTask.getState()); assertEquals(3, mockTask.getAttemptList().size()); } @@ -964,35 +964,35 @@ public void testKilledAttemptUpdatesDAGScheduler() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Have the first task succeed eventHandler.events.clear(); - mockTask.handle(createTaskTASucceededEvent(firstAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(firstAttempt.getTaskAttemptID())); verifyOutgoingEvents(eventHandler.events, DAGEventType.DAG_SCHEDULER_UPDATE, VertexEventType.V_TASK_COMPLETED, VertexEventType.V_TASK_ATTEMPT_COMPLETED); // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(firstAttempt.getID().getId())); + eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); assertEquals(TaskAttemptEventType.TA_KILL_REQUEST, event.getType()); - assertEquals(specAttempt.getID(), + assertEquals(specAttempt.getTaskAttemptID(), ((TaskAttemptEventKillRequest) event).getTaskAttemptID()); eventHandler.events.clear(); // Emulate the spec attempt being killed - mockTask.handle(createTaskTAKilledEvent(specAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(specAttempt.getTaskAttemptID())); assertTaskSucceededState(); verifyOutgoingEvents(eventHandler.events, DAGEventType.DAG_SCHEDULER_UPDATE, VertexEventType.V_TASK_ATTEMPT_COMPLETED); @@ -1003,32 +1003,32 @@ public void testSpeculatedThenRetroactiveFailure() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Have the first task succeed eventHandler.events.clear(); - mockTask.handle(createTaskTASucceededEvent(firstAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(firstAttempt.getTaskAttemptID())); // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(firstAttempt.getID().getId())); + eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); assertEquals(TaskAttemptEventType.TA_KILL_REQUEST, event.getType()); - assertEquals(specAttempt.getID(), + assertEquals(specAttempt.getTaskAttemptID(), ((TaskAttemptEventKillRequest) event).getTaskAttemptID()); // Emulate the spec attempt being killed - mockTask.handle(createTaskTAKilledEvent(specAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(specAttempt.getTaskAttemptID())); assertTaskSucceededState(); // Now fail the attempt after it has succeeded @@ -1039,7 +1039,7 @@ public void testSpeculatedThenRetroactiveFailure() { TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, mockTezEvent, 1); eventHandler.events.clear(); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID(), TaskFailureType.NON_FATAL, outputFailedEvent)); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent)); // The task should still be in the scheduled state assertTaskScheduledState(); @@ -1059,15 +1059,15 @@ public void testIgnoreSpeculationOnSuccessfulOriginalAttempt() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); // Mock success of the first task attempt updateAttemptState(firstAttempt, TaskAttemptState.SUCCEEDED); - firstAttempt.handle(new TaskAttemptEvent(firstAttempt.getID(), TaskAttemptEventType.TA_DONE)); + firstAttempt.handle(new TaskAttemptEvent(firstAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); // Verify the speculation scheduling is ignored and no speculative attempt was added to the task - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); assertEquals(1, mockTask.getAttemptList().size()); } @@ -1076,15 +1076,15 @@ public void testIgnoreSpeculationAfterOriginalAttemptCommit() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Mock commit of the first task attempt - mockTask.canCommit(firstAttempt.getID()); + mockTask.canCommit(firstAttempt.getTaskAttemptID()); // Verify the speculation scheduling is ignored and no speculative attempt was added to the task - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); assertEquals(1, mockTask.getAttemptList().size()); } @@ -1094,10 +1094,10 @@ public void testSucceededAttemptStatusWithRetroActiveFailures() throws Interrupt TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1119,8 +1119,8 @@ public void testSucceededAttemptStatusWithRetroActiveFailures() throws Interrupt new TaskAttemptEvent(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), TaskAttemptEventType.TA_DONE)); - mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getID())); - mockTask.handle(new TaskEventTASucceeded(firstMockTaskAttempt.getID())); + mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getTaskAttemptID())); + mockTask.handle(new TaskEventTASucceeded(firstMockTaskAttempt.getTaskAttemptID())); assertTrue("Attempts should have succeeded!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED && secondMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED); @@ -1148,10 +1148,10 @@ public void testFailedAttemptStatus() throws InterruptedException { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1176,12 +1176,12 @@ public void testFailedAttemptStatus() throws InterruptedException { TaskAttemptTerminationCause.NO_PROGRESS)); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); secondMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - secondMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); - mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + secondMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); assertTrue("Attempts should have failed!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED @@ -1204,10 +1204,10 @@ public void testSucceededLeafTaskWithRetroFailures() throws InterruptedException TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1230,18 +1230,18 @@ public void testSucceededLeafTaskWithRetroFailures() throws InterruptedException TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "test", TaskAttemptTerminationCause.CONTAINER_EXITED)); - mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getID())); + mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getTaskAttemptID())); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); InputReadErrorEvent mockReEvent = InputReadErrorEvent.create("", 0, 0); - TezTaskAttemptID mockDestId = firstMockTaskAttempt.getID(); + TezTaskAttemptID mockDestId = firstMockTaskAttempt.getTaskAttemptID(); EventMetaData meta = new EventMetaData(EventProducerConsumerType.INPUT, "Vertex", "Edge", mockDestId); TezEvent tzEvent = new TezEvent(mockReEvent, meta); TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, tzEvent, 1); firstMockTaskAttempt.handle(outputFailedEvent); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); Assert.assertEquals(mockTask.getInternalState(), TaskStateInternal.SUCCEEDED); } @@ -1255,7 +1255,7 @@ private void failAttempt(MockTaskAttemptImpl taskAttempt, int index, int expecte new TaskAttemptEventOutputFailed(mockDestId, tzEvent, 1); taskAttempt.handle( outputFailedEvent); - TaskEvent tEventFail1 = new TaskEventTAFailed(taskAttempt.getID(), TaskFailureType.NON_FATAL, outputFailedEvent); + TaskEvent tEventFail1 = new TaskEventTAFailed(taskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent); mockTask.handle(tEventFail1); assertEquals("Unexpected number of incomplete attempts!", expectedIncompleteAttempts, mockTask.getUncompletedAttemptsCount()); @@ -1274,10 +1274,10 @@ public void testFailedTaskTransitionWithLaunchedAttempt() throws InterruptedExce TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1302,21 +1302,21 @@ public void testFailedTaskTransitionWithLaunchedAttempt() throws InterruptedExce TaskAttemptTerminationCause.NO_PROGRESS)); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); secondMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - secondMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); - mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + secondMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); assertTrue("Attempts should have failed!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED && secondMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED); assertEquals("Task should have no uncompleted attempts!", 0, mockTask.getUncompletedAttemptsCount()); assertTrue("Task should have failed!", mockTask.getState() == TaskState.FAILED); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl thirdMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getID())); + mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getTaskAttemptID())); } @Test (timeout = 30000) @@ -1324,10 +1324,10 @@ public void testKilledTaskTransitionWithLaunchedAttempt() throws InterruptedExce TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1342,30 +1342,30 @@ public void testKilledTaskTransitionWithLaunchedAttempt() throws InterruptedExce new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()))); firstMockTaskAttempt.handle( new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()))); - mockTask.handle(new TaskEventTermination(mockTask.getTaskId(), + mockTask.handle(new TaskEventTermination(mockTask.getTaskID(), TaskAttemptTerminationCause.FRAMEWORK_ERROR, "test")); secondMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); - mockTask.handle(new TaskEventTAKilled(secondMockTaskAttempt.getID(), - new TaskAttemptEvent(secondMockTaskAttempt.getID(), TaskAttemptEventType.TA_KILLED))); + mockTask.handle(new TaskEventTAKilled(secondMockTaskAttempt.getTaskAttemptID(), + new TaskAttemptEvent(secondMockTaskAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_KILLED))); firstMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); - mockTask.handle(new TaskEventTAKilled(firstMockTaskAttempt.getID(), - new TaskAttemptEvent(firstMockTaskAttempt.getID(), TaskAttemptEventType.TA_KILLED))); + mockTask.handle(new TaskEventTAKilled(firstMockTaskAttempt.getTaskAttemptID(), + new TaskAttemptEvent(firstMockTaskAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_KILLED))); firstMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); assertEquals("Task should have been killed!", mockTask.getInternalState(), TaskStateInternal.KILLED); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl thirdMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getID())); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getTaskAttemptID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl fourthMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTASucceededEvent(fourthMockTaskAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(fourthMockTaskAttempt.getTaskAttemptID())); MockTaskAttemptImpl fifthMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTAFailedEvent(fifthMockTaskAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(fifthMockTaskAttempt.getTaskAttemptID())); } // TODO Add test to validate the correct commit attempt. @@ -1418,7 +1418,7 @@ public MockTaskImpl(TezVertexID vertexId, int partition, @Override protected TaskAttemptImpl createAttempt(int attemptNumber, TezTaskAttemptID schedCausalTA) { MockTaskAttemptImpl attempt = new MockTaskAttemptImpl( - TezBuilderUtils.newTaskAttemptId(getTaskId(), attemptNumber), + TezBuilderUtils.newTaskAttemptId(getTaskID(), attemptNumber), eventHandler, taskCommunicatorManagerInterface, conf, clock, taskHeartbeatHandler, appContext, true, taskResource, containerContext, schedCausalTA); 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 fb7872ff9a..5cdcf49206 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 @@ -386,7 +386,7 @@ private class TaskAttemptEventDispatcher implements EventHandler)task.getAttempt( event.getTaskAttemptID())).handle(event); @@ -399,7 +399,7 @@ private class TaskEventDispatcher implements EventHandler { @Override public void handle(TaskEvent event) { events.add(event); - VertexImpl vertex = vertexIdMap.get(event.getTaskID().getVertexID()); + VertexImpl vertex = vertexIdMap.get(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); if (task != null) { ((EventHandler)task).handle(event); @@ -437,7 +437,7 @@ private class VertexEventDispatcher @Override public void handle(VertexEvent event) { - VertexImpl vertex = vertexIdMap.get(event.getVertexId()); + VertexImpl vertex = vertexIdMap.get(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -3155,7 +3155,7 @@ private void checkTasks(Vertex v, int numTasks) { int i = 0; // iteration maintains order due to linked hash map for(Task task : tasks.values()) { - Assert.assertEquals(i, task.getTaskId().getId()); + Assert.assertEquals(i, task.getTaskID().getId()); i++; } } @@ -3638,7 +3638,7 @@ public void testVertexTaskAttemptProcessorFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3652,10 +3652,10 @@ public void testVertexTaskAttemptProcessorFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.APPLICATION_ERROR)); dispatcher.await(); @@ -3673,7 +3673,7 @@ public void testVertexTaskAttemptInputFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3687,11 +3687,11 @@ public void testVertexTaskAttemptInputFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.INPUT_READ_ERROR)); dispatcher.await(); @@ -3710,7 +3710,7 @@ public void testVertexTaskAttemptOutputFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3724,11 +3724,11 @@ public void testVertexTaskAttemptOutputFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.OUTPUT_WRITE_ERROR)); dispatcher.await(); @@ -4800,7 +4800,7 @@ public void testInputInitializerEventsMultipleSources() throws Exception { 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(); + TezTaskID taskId = task.getTaskID(); TezTaskAttemptID attemptId = TezTaskAttemptID.getInstance(taskId, 0); int numEventsFromTask = taskId.getId() + 1; for (int i = 0; i < numEventsFromTask; i++) { @@ -5499,7 +5499,7 @@ public void testVertexWithInitializerSuccess() throws Exception { for (int i=0; i()); //Vertex 1, Task 4, Attempt 1, host1 TezTaskAttemptID taID114 = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexID11, 4), 1); TaskAttempt ta114 = mock(TaskAttempt.class); - doReturn(taID114).when(ta114).getID(); + doReturn(taID114).when(ta114).getTaskAttemptID(); doReturn("Mock for TA " + taID114.toString()).when(ta114).toString(); AMSchedulerEventTALaunchRequest lrEvent14 = createLaunchRequestEvent( taID114, ta114, resource1, host1, racks, priority1, new HashMap()); @@ -1261,7 +1261,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc //Vertex 2, Task 1, Attempt 1, host1, lr2 TezTaskAttemptID taID211 = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexID21, 1), 1); TaskAttempt ta211 = mock(TaskAttempt.class); - doReturn(taID211).when(ta211).getID(); + doReturn(taID211).when(ta211).getTaskAttemptID(); doReturn("Mock for TA " + taID211.toString()).when(ta211).toString(); AMSchedulerEventTALaunchRequest lrEvent21 = createLaunchRequestEvent(taID211, ta211, resource1, host1, racks, priority1, v21LR); @@ -1297,7 +1297,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc TezTaskAttemptID taID311 = TezTaskAttemptID.getInstance( TezTaskID.getInstance(TezVertexID.getInstance(dagID3, 1), 1), 1); TaskAttempt ta311 = mock(TaskAttempt.class); - doReturn(taID311).when(ta311).getID(); + doReturn(taID311).when(ta311).getTaskAttemptID(); doReturn("Mock for TA " + taID311).when(ta311).toString(); AMSchedulerEventTALaunchRequest lrEvent31 = createLaunchRequestEvent(taID311, ta311, resource1, host1, racks, priority1, v31LR); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index dcf9a5dd69..3ddd1861d4 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -189,7 +189,7 @@ public void testSimpleAllocate() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); Resource resource = Resource.newInstance(1024, 1); ContainerContext containerContext = new ContainerContext(new HashMap(), new Credentials(), @@ -228,7 +228,7 @@ public void testTASucceededAfterContainerCleanup() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); Resource resource = Resource.newInstance(1024, 1); ContainerContext containerContext = new ContainerContext(new HashMap(), new Credentials(), @@ -268,7 +268,7 @@ public void testTAUnsuccessfulAfterContainerCleanup() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); ContainerId mockCId = mock(ContainerId.class); Container container = mock(Container.class); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java index 16b391b55f..94c8df6233 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java @@ -640,7 +640,7 @@ Map getTasksTestHelper(List tasks, List > t //Creating mock tasks and attaching to mock vertex Map taskMap = Maps.newHashMap(); for(Task task : tasks) { - TezTaskID taskId = task.getTaskId(); + TezTaskID taskId = task.getTaskID(); int taskIndex = taskId.getId(); doReturn(task).when(mockVertex).getTask(taskIndex); taskMap.put(taskId, task); @@ -688,7 +688,7 @@ private List createMockTasks() { private Task createMockTask(String taskIDStr, TaskState status, float progress) { Task mockTask = mock(Task.class); - doReturn(TezTaskID.fromString(taskIDStr)).when(mockTask).getTaskId(); + doReturn(TezTaskID.fromString(taskIDStr)).when(mockTask).getTaskID(); doReturn(status).when(mockTask).getState(); doReturn(progress).when(mockTask).getProgress(); @@ -711,7 +711,7 @@ private Task createMockTask(String taskIDStr, TaskState status, float progress) private void verifySingleTaskResult(Task mockTask, Map taskResult) { Assert.assertEquals(3, taskResult.size()); - Assert.assertEquals(mockTask.getTaskId().toString(), taskResult.get("id")); + Assert.assertEquals(mockTask.getTaskID().toString(), taskResult.get("id")); Assert.assertEquals(mockTask.getState().toString(), taskResult.get("status")); Assert.assertEquals(Float.toString(mockTask.getProgress()), taskResult.get("progress")); } @@ -777,7 +777,7 @@ Map getAttemptsTestHelper(List attempts, List getAttemptsTestHelper(List attempts, List attemptsMap = Maps.newHashMap(); for(TaskAttempt attempt : attempts) { - TezTaskAttemptID attemptId = attempt.getID(); + TezTaskAttemptID attemptId = attempt.getTaskAttemptID(); doReturn(attempt).when(mockTask).getAttempt(attemptId); attemptsMap.put(attemptId, attempt); } @@ -835,7 +835,7 @@ private List createMockAttempts() { private TaskAttempt createMockAttempt(String attemptIDStr, TaskAttemptState status, float progress) { TaskAttempt mockAttempt = mock(TaskAttempt.class); - doReturn(TezTaskAttemptID.fromString(attemptIDStr)).when(mockAttempt).getID(); + doReturn(TezTaskAttemptID.fromString(attemptIDStr)).when(mockAttempt).getTaskAttemptID(); doReturn(status).when(mockAttempt).getState(); doReturn(progress).when(mockAttempt).getProgress(); @@ -858,7 +858,7 @@ private TaskAttempt createMockAttempt(String attemptIDStr, TaskAttemptState stat private void verifySingleAttemptResult(TaskAttempt mockTask, Map taskResult) { Assert.assertEquals(3, taskResult.size()); - Assert.assertEquals(mockTask.getID().toString(), taskResult.get("id")); + Assert.assertEquals(mockTask.getTaskAttemptID().toString(), taskResult.get("id")); Assert.assertEquals(mockTask.getState().toString(), taskResult.get("status")); Assert.assertEquals(Float.toString(mockTask.getProgress()), taskResult.get("progress")); } 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 50a80cb5f5..51b4bf9f92 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 @@ -186,8 +186,8 @@ private void testDAGSubmittedEvent() throws Exception { testProtoConversion(event); Assert.assertEquals(event.getApplicationAttemptId(), deserializedEvent.getApplicationAttemptId()); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getDAGName(), deserializedEvent.getDAGName()); Assert.assertEquals(event.getSubmitTime(), @@ -204,8 +204,8 @@ private void testDAGInitializedEvent() throws Exception { "user", "dagName", null); DAGInitializedEvent deserializedEvent = (DAGInitializedEvent) testProtoConversion(event); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getInitTime(), deserializedEvent.getInitTime()); logEvents(event, deserializedEvent); } @@ -216,8 +216,8 @@ private void testDAGStartedEvent() throws Exception { "user", "dagName"); DAGStartedEvent deserializedEvent = (DAGStartedEvent) testProtoConversion(event); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getStartTime(), deserializedEvent.getStartTime()); logEvents(event, deserializedEvent); } @@ -243,8 +243,8 @@ private void testDAGFinishedEvent() throws Exception { DAGFinishedEvent deserializedEvent = (DAGFinishedEvent) testProtoConversion(event); Assert.assertEquals( - event.getDagID(), - deserializedEvent.getDagID()); + event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getState(), deserializedEvent.getState()); Assert.assertNotEquals(event.getStartTime(), deserializedEvent.getStartTime()); Assert.assertEquals(event.getFinishTime(), deserializedEvent.getFinishTime()); @@ -264,8 +264,8 @@ private void testDAGFinishedEvent() throws Exception { DAGFinishedEvent deserializedEvent = (DAGFinishedEvent) testProtoConversion(event); Assert.assertEquals( - event.getDagID(), - deserializedEvent.getDagID()); + event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getState(), deserializedEvent.getState()); Assert.assertNotEquals(event.getStartTime(), deserializedEvent.getStartTime()); Assert.assertEquals(event.getFinishTime(), deserializedEvent.getFinishTime()); diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java index eef73a0d0a..7864e1c852 100644 --- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java +++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java @@ -556,12 +556,12 @@ private void checkAndThrowExceptionForTests(SubmitWorkRequestProto request) thro } TaskSpec taskSpec = ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()); - if (taskSpec.getTaskAttemptID().getTaskID().getId() == 0 && + if (taskSpec.getTaskID().getId() == 0 && taskSpec.getTaskAttemptID().getId() == 0) { LOG.info("Simulating Rejected work"); throw new RejectedExecutionException( "Simulating Rejected work for taskAttemptId=" + taskSpec.getTaskAttemptID()); - } else if (taskSpec.getTaskAttemptID().getTaskID().getId() == 1 && + } else if (taskSpec.getTaskID().getId() == 1 && taskSpec.getTaskAttemptID().getId() == 0) { LOG.info("Simulating Task Setup Failure during launch"); throw new TezException("Simulating Task Setup Failure during launch for taskAttemptId=" + diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java index 0f1b56d3d2..2fddd00522 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java @@ -39,7 +39,7 @@ public static JobID toMRJobId(TezDAGID dagId) { public static TaskID toMRTaskId(TezTaskID taskid) { return new TaskID( - toMRJobId(taskid.getVertexID().getDAGId()), + toMRJobId(taskid.getDAGID()), taskid.getVertexID().getId() == 0 ? TaskType.MAP : TaskType.REDUCE, taskid.getId()); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java index e5e7022064..941c8732bd 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java @@ -53,8 +53,8 @@ public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptID( public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptIDFromTezTaskAttemptId(TezTaskAttemptID tezTaId, boolean isMap) { - TezVertexID vId = tezTaId.getTaskID().getVertexID(); - ApplicationId appId = vId.getDAGId().getApplicationId(); + TezVertexID vId = tezTaId.getVertexID(); + ApplicationId appId = vId.getApplicationId(); return new org.apache.hadoop.mapred.TaskAttemptID( new org.apache.hadoop.mapred.TaskID(String.valueOf(appId.getClusterTimestamp()) + String.valueOf(vId.getId()), appId.getId(), @@ -65,7 +65,7 @@ public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptID( public static org.apache.hadoop.mapred.TaskID createMockTaskAttemptIDFromTezTaskId(TezTaskID tezTaId, boolean isMap) { TezVertexID vId = tezTaId.getVertexID(); - ApplicationId appId = vId.getDAGId().getApplicationId(); + ApplicationId appId = vId.getApplicationId(); return new org.apache.hadoop.mapred.TaskID(String.valueOf(appId.getClusterTimestamp()) + String.valueOf(vId.getId()), appId.getId(), isMap ? TaskType.MAP : TaskType.REDUCE, tezTaId.getId()); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java index 1a13168cb7..55e5709c3b 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java @@ -36,7 +36,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputBuffer; @@ -321,8 +320,7 @@ public void initTask(LogicalOutput output) throws IOException, } this.mrReporter = new MRTaskReporter(processorContext); this.useNewApi = jobConf.getUseNewMapper(); - TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getTaskID() - .getVertexID().getDAGId(); + TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getDAGID(); this.jobContext = new JobContextImpl(jobConf, dagId, mrReporter); this.taskAttemptContext = diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java index c1711ce2cb..b57f4f0e65 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java @@ -264,8 +264,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_VERTEX_ID: String vertexName = entity; TezVertexID tezVertexID = TezVertexID.fromString(vertexName); - if (!tezDAGID.equals(tezVertexID.getDAGId())) { - LOG.warn("{} does not belong to {} ('{}' != '{}')}", vertexName, tezDAGID, tezDAGID, tezVertexID.getDAGId()); + if (!tezDAGID.equals(tezVertexID.getDAGID())) { + LOG.warn("{} does not belong to {} ('{}' != '{}')}", vertexName, tezDAGID, tezDAGID, tezVertexID.getDAGID()); continue; } if (!vertexJsonMap.containsKey(vertexName)) { @@ -278,9 +278,9 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_TASK_ID: String taskName = entity; TezTaskID tezTaskID = TezTaskID.fromString(taskName); - if (!tezDAGID.equals(tezTaskID.getVertexID().getDAGId())) { + if (!tezDAGID.equals(tezTaskID.getDAGID())) { LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskName, tezDAGID, tezDAGID, - tezTaskID.getVertexID().getDAGId()); + tezTaskID.getDAGID()); continue; } if (!taskJsonMap.containsKey(taskName)) { @@ -293,9 +293,9 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_TASK_ATTEMPT_ID: String taskAttemptName = entity; TezTaskAttemptID tezAttemptId = TezTaskAttemptID.fromString(taskAttemptName); - if (!tezDAGID.equals(tezAttemptId.getTaskID().getVertexID().getDAGId())) { + if (!tezDAGID.equals(tezAttemptId.getDAGID())) { LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskAttemptName, tezDAGID, tezDAGID, - tezAttemptId.getTaskID().getVertexID().getDAGId()); + tezAttemptId.getDAGID()); continue; } if (!attemptJsonMap.containsKey(taskAttemptName)) { diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java index af8e292b24..9f3881c8b2 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java @@ -90,9 +90,8 @@ protected void linkParsedContents() { //Link task to task attempt TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(attemptInfo .getTaskAttemptId()); - VertexInfo vertexInfo = dagInfo.getVertexFromId(taskAttemptId.getTaskID() - .getVertexID().toString()); - Preconditions.checkState(vertexInfo != null, "Vertex " + taskAttemptId.getTaskID() + VertexInfo vertexInfo = dagInfo.getVertexFromId(taskAttemptId.getVertexID().toString()); + Preconditions.checkState(vertexInfo != null, "Vertex " + taskAttemptId .getVertexID().toString() + " is not present in DAG"); TaskInfo taskInfo = vertexInfo.getTask(taskAttemptId.getTaskID().toString()); attemptInfo.setTaskInfo(taskInfo); diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java index 09079bdf1c..f5f436eb9d 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -150,7 +150,7 @@ private HistoryEventProto.Builder makeBuilderForEvent(HistoryEvent event, long t } if (vertexId != null) { builder.setVertexId(vertexId.toString()); - dagId = vertexId.getDAGId(); + dagId = vertexId.getDAGID(); } if (dagId != null) { builder.setDagId(dagId.toString()); @@ -240,7 +240,7 @@ private HistoryEventProto convertContainerStoppedEvent(ContainerStoppedEvent eve private HistoryEventProto convertDAGSubmittedEvent(DAGSubmittedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getSubmitTime(), - event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getDAGID(), null, event.getApplicationAttemptId(), null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDAGName()); if (event.getDAGPlan().hasCallerContext() && @@ -267,11 +267,11 @@ private HistoryEventProto convertDAGSubmittedEvent(DAGSubmittedEvent event) { private HistoryEventProto convertDAGInitializedEvent(DAGInitializedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getInitTime(), - event.getDagID(), null, null, null, null, null, event.getUser()); + event.getDAGID(), null, null, null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); if (event.getVertexNameIDMap() != null) { - Map nameIdStrMap = new TreeMap(); + Map nameIdStrMap = new TreeMap<>(); for (Entry entry : event.getVertexNameIDMap().entrySet()) { nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); } @@ -282,7 +282,7 @@ private HistoryEventProto convertDAGInitializedEvent(DAGInitializedEvent event) private HistoryEventProto convertDAGStartedEvent(DAGStartedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), - event.getDagID(), null, null, null, null, null, event.getUser()); + event.getDAGID(), null, null, null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); addEventData(builder, ATSConstants.STATUS, event.getDagState().name()); @@ -292,7 +292,7 @@ private HistoryEventProto convertDAGStartedEvent(DAGStartedEvent event) { private HistoryEventProto convertDAGFinishedEvent(DAGFinishedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getFinishTime(), - event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getDAGID(), null, event.getApplicationAttemptId(), null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); if (event.getDAGPlan().hasCallerContext()) { @@ -473,7 +473,7 @@ private HistoryEventProto convertVertexReconfigureDoneEvent(VertexConfigurationD HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getReconfigureDoneTime(), null, null, null, event.getVertexID(), null, null, null); if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) { - Map updatedEdgeManagers = new HashMap(); + Map updatedEdgeManagers = new HashMap<>(); for (Entry entry : event.getSourceEdgeProperties().entrySet()) { updatedEdgeManagers.put(entry.getKey(), DAGUtils.convertEdgeProperty(entry.getValue())); } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java index 008b05d616..d675a659a0 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -73,7 +73,7 @@ public ProtoHistoryLoggingService() { } @Override - protected void serviceInit(Configuration conf) throws Exception { + protected void serviceInit(Configuration conf) { LOG.info("Initing ProtoHistoryLoggingService"); setConfig(conf); loggingDisabled = !conf.getBoolean(TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED, @@ -146,7 +146,7 @@ private void loop() { } catch (InterruptedException e) { LOG.info("EventQueue poll interrupted, ignoring it.", e); } catch (IOException e) { - TezDAGID dagid = evt.getDagID(); + TezDAGID dagid = evt.getDAGID(); HistoryEventType type = evt.getHistoryEvent().getEventType(); // Retry is hard, because there are several places where this exception can happen // the state will get messed up a lot. @@ -160,7 +160,7 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { return; } HistoryEvent historyEvent = event.getHistoryEvent(); - if (event.getDagID() == null) { + if (event.getDAGID() == null) { if (historyEvent.getEventType() == HistoryEventType.APP_LAUNCHED) { appEventsFile = appEventsWriter.getPath().toString(); appLaunchedEventOffset = appEventsWriter.getOffset(); @@ -168,7 +168,7 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { appEventsWriter.writeProto(converter.convert(historyEvent)); } else { HistoryEventType type = historyEvent.getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (type == HistoryEventType.DAG_FINISHED) { finishCurrentDag((DAGFinishedEvent)historyEvent); } else if (type == HistoryEventType.DAG_SUBMITTED) { @@ -220,7 +220,7 @@ private void finishCurrentDag(DAGFinishedEvent event) throws IOException { .setAppLaunchedEventOffset(appLaunchedEventOffset) .setWriteTime(System.currentTimeMillis()); if (event != null) { - entry.setDagId(event.getDagID().toString()); + entry.setDagId(event.getDAGID().toString()); } manifestEventsWriter.writeProto(entry.build()); manifestEventsWriter.hflush(); diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java index d211feb0e3..2ee2c21b79 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java @@ -94,18 +94,18 @@ private Set convertToTimelineEntityGroupIds(String entity } else if (entityType.equals(EntityTypes.TEZ_VERTEX_ID.name())) { TezVertexID vertexID = TezVertexID.fromString(entityId); if (vertexID != null) { - return createTimelineEntityGroupIds(vertexID.getDAGId()); + return createTimelineEntityGroupIds(vertexID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_TASK_ID.name())) { TezTaskID taskID = TezTaskID.fromString(entityId); if (taskID != null) { - return createTimelineEntityGroupIds(taskID.getVertexID().getDAGId()); + return createTimelineEntityGroupIds(taskID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_TASK_ATTEMPT_ID.name())) { TezTaskAttemptID taskAttemptID = TezTaskAttemptID.fromString(entityId); if (taskAttemptID != null) { - return createTimelineEntityGroupIds(taskAttemptID.getTaskID().getVertexID().getDAGId()); + return createTimelineEntityGroupIds(taskAttemptID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_CONTAINER_ID.name())) { String cId = entityId; diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java index f79a78b9fa..f284fe4049 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java @@ -451,7 +451,7 @@ public void testDagLoggingDisabled() throws Exception { DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent); historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent)); Thread.sleep(1000l); - String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDagID(); + String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDAGID(); Client client = new Client(); WebResource resource = client.resource(url); @@ -496,7 +496,7 @@ public void testDagLoggingEnabled() throws Exception { DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent); historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent)); Thread.sleep(1000l); - String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDagID(); + String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDAGID(); Client client = new Client(); WebResource resource = client.resource(url); diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java index a71f0d8db0..5068fb77db 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java @@ -311,9 +311,9 @@ public TimelineEntityGroupId getGroupId(DAGHistoryEvent event) { case VERTEX_GROUP_COMMIT_FINISHED: case DAG_RECOVERED: String entityGroupId = numDagsPerGroup > 1 - ? event.getDagID().getGroupId(numDagsPerGroup) - : event.getDagID().toString(); - return TimelineEntityGroupId.newInstance(event.getDagID().getApplicationId(), entityGroupId); + ? event.getDAGID().getGroupId(numDagsPerGroup) + : event.getDAGID().toString(); + return TimelineEntityGroupId.newInstance(event.getApplicationId(), entityGroupId); case APP_LAUNCHED: case AM_LAUNCHED: case AM_STARTED: @@ -333,7 +333,7 @@ public void handle(DAGHistoryEvent event) { private boolean isValidEvent(DAGHistoryEvent event) { HistoryEventType eventType = event.getHistoryEvent().getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = @@ -373,7 +373,7 @@ private boolean isValidEvent(DAGHistoryEvent event) { private void handleEvents(DAGHistoryEvent event) { String domainId = getDomainForEvent(event); // skippedDags is updated in the above call so check again. - if (event.getDagID() != null && skippedDAGs.contains(event.getDagID())) { + if (event.getDAGID() != null && skippedDAGs.contains(event.getDAGID())) { return; } TimelineEntityGroupId groupId = getGroupId(event); @@ -417,7 +417,7 @@ private String getDomainForEvent(DAGHistoryEvent event) { return domainId; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryEvent historyEvent = event.getHistoryEvent(); if (dagId == null || !HistoryEventType.isDAGSpecificEvent(historyEvent.getEventType())) { return domainId; diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java index 6d035cce2c..c1883a9216 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -291,7 +291,7 @@ public void handle(DAGHistoryEvent event) { private boolean isValidEvent(DAGHistoryEvent event) { HistoryEventType eventType = event.getHistoryEvent().getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = @@ -333,7 +333,7 @@ private void handleEvents(List events) { for (DAGHistoryEvent event : events) { String domainId = getDomainForEvent(event); // skippedDags is updated in the above call so check again. - if (event.getDagID() != null && skippedDAGs.contains(event.getDagID())) { + if (event.getDAGID() != null && skippedDAGs.contains(event.getDAGID())) { continue; } List eventEntities = HistoryEventTimelineConversion.convertToTimelineEntities( @@ -377,7 +377,7 @@ private String getDomainForEvent(DAGHistoryEvent event) { return domainId; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryEvent historyEvent = event.getHistoryEvent(); if (dagId == null || !HistoryEventType.isDAGSpecificEvent(historyEvent.getEventType())) { return domainId; diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java index c40d3a8f78..b9457a0761 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java @@ -32,7 +32,6 @@ import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; -import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.api.records.DAGProtos.CallerContextProto; import org.apache.tez.dag.app.web.AMWebController; import org.apache.tez.dag.history.HistoryEvent; @@ -288,7 +287,7 @@ private static TimelineEntity convertContainerStoppedEvent(ContainerStoppedEvent private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent finishEvt = new TimelineEvent(); @@ -298,7 +297,7 @@ private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addPrimaryFilter(ATSConstants.STATUS, event.getState().name()); if (event.getDAGPlan().hasCallerContext() @@ -327,10 +326,10 @@ private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { private static TimelineEntity convertDAGFinishedToDAGExtraInfoEntity(DAGFinishedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_EXTRA_INFO.name()); - atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDagID().toString()); + atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDAGID().toString()); TimelineEvent submitEvt = new TimelineEvent(); submitEvt.setEventType(HistoryEventType.DAG_FINISHED.name()); @@ -345,7 +344,7 @@ private static TimelineEntity convertDAGFinishedToDAGExtraInfoEntity(DAGFinished private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent initEvt = new TimelineEvent(); @@ -355,13 +354,13 @@ private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent eve atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addOtherInfo(ATSConstants.INIT_TIME, event.getInitTime()); if (event.getVertexNameIDMap() != null) { - Map nameIdStrMap = new TreeMap(); + Map nameIdStrMap = new TreeMap<>(); for (Entry entry : event.getVertexNameIDMap().entrySet()) { nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); } @@ -373,7 +372,7 @@ private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent eve private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent startEvt = new TimelineEvent(); @@ -383,7 +382,7 @@ private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addOtherInfo(ATSConstants.START_TIME, event.getStartTime()); @@ -394,7 +393,7 @@ private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_APPLICATION.name(), @@ -412,7 +411,7 @@ private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDAGName()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); if (event.getDAGPlan().hasCallerContext() && event.getDAGPlan().getCallerContext().hasCallerId()) { @@ -450,10 +449,10 @@ private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) private static TimelineEntity convertDAGSubmittedToDAGExtraInfoEntity(DAGSubmittedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_EXTRA_INFO.name()); - atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDagID().toString()); + atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDAGID().toString()); TimelineEvent submitEvt = new TimelineEvent(); submitEvt.setEventType(HistoryEventType.DAG_SUBMITTED.name()); @@ -477,13 +476,13 @@ private static TimelineEntity convertTaskAttemptFinishedEvent(TaskAttemptFinishe atsEntity.setEntityType(EntityTypes.TEZ_TASK_ATTEMPT_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.TASK_ATTEMPT_FINISHED.name()); @@ -542,16 +541,16 @@ private static TimelineEntity convertTaskAttemptStartedEvent(TaskAttemptStartedE atsEntity.setStartTime(event.getStartTime()); atsEntity.addRelatedEntity(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.TASK_ATTEMPT_STARTED.name()); @@ -579,11 +578,11 @@ private static TimelineEntity convertTaskFinishedEvent(TaskFinishedEvent event) atsEntity.setEntityType(EntityTypes.TEZ_TASK_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.TASK_FINISHED.name()); @@ -614,14 +613,14 @@ private static TimelineEntity convertTaskStartedEvent(TaskStartedEvent event) { atsEntity.setEntityType(EntityTypes.TEZ_TASK_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.TASK_STARTED.name()); @@ -643,9 +642,9 @@ private static TimelineEntity convertVertexFinishedEvent(VertexFinishedEvent eve atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.VERTEX_FINISHED.name()); @@ -685,12 +684,12 @@ private static TimelineEntity convertVertexInitializedEvent(VertexInitializedEve atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent initEvt = new TimelineEvent(); initEvt.setEventType(HistoryEventType.VERTEX_INITIALIZED.name()); @@ -718,9 +717,9 @@ private static TimelineEntity convertVertexStartedEvent(VertexStartedEvent event atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.VERTEX_STARTED.name()); @@ -741,17 +740,17 @@ private static TimelineEntity convertVertexReconfigureDoneEvent( atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent updateEvt = new TimelineEvent(); updateEvt.setEventType(HistoryEventType.VERTEX_CONFIGURE_DONE.name()); updateEvt.setTimestamp(event.getReconfigureDoneTime()); - Map eventInfo = new HashMap(); + Map eventInfo = new HashMap<>(); if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) { - Map updatedEdgeManagers = new HashMap(); + Map updatedEdgeManagers = new HashMap<>(); for (Entry entry : event.getSourceEdgeProperties().entrySet()) { updatedEdgeManagers.put(entry.getKey(), diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java index 0542b33d03..abe8c9dd53 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java @@ -28,6 +28,7 @@ import javax.annotation.Nullable; import org.apache.hadoop.io.Writable; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.util.StringInterner; @@ -35,7 +36,7 @@ * Class that encapsulates all the information to identify the unique * object that either generated an Event or is the recipient of an Event. */ -public class EventMetaData implements Writable { +public class EventMetaData implements Writable, TaskAttemptIDAware { public static enum EventProducerConsumerType { INPUT, @@ -83,6 +84,7 @@ public EventProducerConsumerType getEventGenerator() { return producerConsumerType; } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptID; } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java index 63c251c054..15e756b853 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java @@ -30,12 +30,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; import org.apache.tez.dag.api.ProcessorDescriptor; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.util.StringInterner; import com.google.common.collect.Lists; -public class TaskSpec implements Writable { +public class TaskSpec implements Writable, TaskAttemptIDAware { private TezTaskAttemptID taskAttemptId; private String dagName; @@ -128,7 +129,7 @@ public String getDAGName() { } public int getDagIdentifier() { - return taskAttemptId.getTaskID().getVertexID().getDAGId().getId(); + return taskAttemptId.getDAGID().getId(); } public int getVertexParallelism() { @@ -139,6 +140,7 @@ public String getVertexName() { return vertexName; } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptId; } @@ -266,5 +268,4 @@ public String toString() { } return sb.toString(); } - } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java index a47dac1e0a..b645346654 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java @@ -118,8 +118,7 @@ public TezTaskContextImpl(Configuration conf, String[] workDirs, int appAttemptN @Override public ApplicationId getApplicationId() { - return taskAttemptID.getTaskID().getVertexID().getDAGId() - .getApplicationId(); + return taskAttemptID.getApplicationId(); } @Override @@ -154,12 +153,12 @@ public String getTaskVertexName() { @Override public int getTaskVertexIndex() { - return taskAttemptID.getTaskID().getVertexID().getId(); + return taskAttemptID.getVertexID().getId(); } @Override public int getDagIdentifier() { - return taskAttemptID.getTaskID().getVertexID().getDAGId().getId(); + return taskAttemptID.getDAGID().getId(); } @Override diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java index c82355a9fa..bc8c2d8391 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -61,7 +61,6 @@ import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.TokenCache; 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.records.DAGProtos; import org.apache.tez.dag.records.TezVertexID; @@ -372,13 +371,12 @@ public URI apply(TezLocalResource input) { private void cleanupOnTaskChanged(ContainerTask containerTask) { Preconditions.checkState(!containerTask.shouldDie()); Preconditions.checkState(containerTask.getTaskSpec() != null); - TezVertexID newVertexID = containerTask.getTaskSpec().getTaskAttemptID().getTaskID() - .getVertexID(); + TezVertexID newVertexID = containerTask.getTaskSpec().getTaskAttemptID().getVertexID(); if (lastVertexID != null) { if (!lastVertexID.equals(newVertexID)) { objectRegistry.clearCache(ObjectRegistryImpl.ObjectLifeCycle.VERTEX); } - if (!lastVertexID.getDAGId().equals(newVertexID.getDAGId())) { + if (!lastVertexID.getDAGID().equals(newVertexID.getDAGID())) { objectRegistry.clearCache(ObjectRegistryImpl.ObjectLifeCycle.DAG); startedInputsMap = HashMultimap.create(); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java index 3ed386e346..186ab7e659 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java +++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java @@ -334,7 +334,7 @@ public boolean match(HistoryEvent incomingEvent) { TaskStartedEvent otherEvent = (TaskStartedEvent) incomingEvent; TaskStartedEvent conditionEvent = (TaskStartedEvent) event; // compare vertexId and taskId - return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId() + return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId() && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId(); } break; @@ -344,7 +344,7 @@ public boolean match(HistoryEvent incomingEvent) { TaskFinishedEvent otherEvent = (TaskFinishedEvent) incomingEvent; TaskFinishedEvent conditionEvent = (TaskFinishedEvent) event; // compare vertexId and taskId - return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId() + return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId() && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId(); } break; @@ -354,9 +354,9 @@ public boolean match(HistoryEvent incomingEvent) { TaskAttemptStartedEvent otherEvent = (TaskAttemptStartedEvent) incomingEvent; TaskAttemptStartedEvent conditionEvent = (TaskAttemptStartedEvent) event; // compare vertexId, taskId & taskAttemptId - return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId() + return otherEvent.getVertexID().getId() + == conditionEvent.getVertexID().getId() + && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId() && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId(); } break; @@ -366,9 +366,9 @@ public boolean match(HistoryEvent incomingEvent) { TaskAttemptFinishedEvent otherEvent = (TaskAttemptFinishedEvent) incomingEvent; TaskAttemptFinishedEvent conditionEvent = (TaskAttemptFinishedEvent) event; // compare vertexId, taskId & taskAttemptId - return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId() + return otherEvent.getVertexID().getId() + == conditionEvent.getVertexID().getId() + && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId() && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId(); } break; 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 6d3ab1c711..7fe3b3acc1 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 @@ -463,9 +463,9 @@ private List findTaskAttemptFinishedEvent( if (historyEvent.getEventType() == HistoryEventType.TASK_ATTEMPT_FINISHED) { TaskAttemptFinishedEvent taFinishedEvent = (TaskAttemptFinishedEvent) historyEvent; - if (taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID() + if (taFinishedEvent.getVertexID() .getId() == vertexId - && taFinishedEvent.getTaskAttemptID().getTaskID().getId() == taskId) { + && taFinishedEvent.getTaskID().getId() == taskId) { resultEvents.add(taFinishedEvent); } }