Return-Path: X-Original-To: apmail-tez-commits-archive@minotaur.apache.org Delivered-To: apmail-tez-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 60C4D17D63 for ; Fri, 11 Sep 2015 00:28:37 +0000 (UTC) Received: (qmail 22834 invoked by uid 500); 11 Sep 2015 00:28:37 -0000 Delivered-To: apmail-tez-commits-archive@tez.apache.org Received: (qmail 22795 invoked by uid 500); 11 Sep 2015 00:28:37 -0000 Mailing-List: contact commits-help@tez.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@tez.apache.org Delivered-To: mailing list commits@tez.apache.org Received: (qmail 22785 invoked by uid 99); 11 Sep 2015 00:28:37 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 11 Sep 2015 00:28:37 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id EEF43E1072; Fri, 11 Sep 2015 00:28:36 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: bikas@apache.org To: commits@tez.apache.org Message-Id: <8dbf7c0b047643b2897df84cd38b26dd@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: tez git commit: TEZ-2807. Log data in the finish event instead of the start event (bikas) Date: Fri, 11 Sep 2015 00:28:36 +0000 (UTC) Repository: tez Updated Branches: refs/heads/master cb0890d58 -> a17a08d31 TEZ-2807. Log data in the finish event instead of the start event (bikas) Project: http://git-wip-us.apache.org/repos/asf/tez/repo Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/a17a08d3 Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/a17a08d3 Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/a17a08d3 Branch: refs/heads/master Commit: a17a08d31367f95455d178bbd5e1f24ac1a84ccb Parents: cb0890d Author: Bikas Saha Authored: Thu Sep 10 17:28:23 2015 -0700 Committer: Bikas Saha Committed: Thu Sep 10 17:28:23 2015 -0700 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 15 ++++--- .../events/TaskAttemptFinishedEvent.java | 39 ++++++++++++++++- .../history/events/TaskAttemptStartedEvent.java | 34 +-------------- .../impl/HistoryEventJsonConversion.java | 10 ++--- tez-dag/src/main/proto/HistoryEvents.proto | 19 ++++---- .../app/dag/impl/TestTaskAttemptRecovery.java | 16 +++---- .../tez/dag/app/dag/impl/TestTaskRecovery.java | 46 ++++++++++---------- .../TestHistoryEventsProtoConversion.java | 26 +++++------ .../impl/TestHistoryEventJsonConversion.java | 4 +- .../ats/HistoryEventTimelineConversion.java | 13 +++--- .../ats/TestHistoryEventTimelineConversion.java | 29 ++++++------ 12 files changed, 131 insertions(+), 121 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 6310134..df41a0f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -7,6 +7,7 @@ Release 0.8.1: Unreleased INCOMPATIBLE CHANGES ALL CHANGES: + TEZ-2807. Log data in the finish event instead of the start event TEZ-2799. SimpleHistoryParser NPE TEZ-2643. Minimize number of empty spills in Pipelined Sorter TEZ-2783. Refactor analyzers to extend TezAnalyzerBase http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java ---------------------------------------------------------------------- diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 003e05f..abcd98d 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 @@ -834,9 +834,6 @@ public class TaskAttemptImpl implements TaskAttempt, { TaskAttemptStartedEvent tEvent = (TaskAttemptStartedEvent) historyEvent; this.launchTime = tEvent.getStartTime(); - this.creationTime = tEvent.getCreationTime(); - this.allocationTime = tEvent.getAllocationTime(); - this.creationCausalTA = tEvent.getCreationCausalTA(); recoveryStartEventSeen = true; recoveredState = TaskAttemptState.RUNNING; this.containerId = tEvent.getContainerId(); @@ -846,7 +843,11 @@ public class TaskAttemptImpl implements TaskAttempt, case TASK_ATTEMPT_FINISHED: { TaskAttemptFinishedEvent tEvent = (TaskAttemptFinishedEvent) historyEvent; + this.creationTime = tEvent.getCreationTime(); + this.allocationTime = tEvent.getAllocationTime(); + this.launchTime = tEvent.getStartTime(); this.finishTime = tEvent.getFinishTime(); + this.creationCausalTA = tEvent.getCreationCausalTA(); this.reportedStatus.counters = tEvent.getCounters(); this.reportedStatus.progress = 1f; this.reportedStatus.state = tEvent.getState(); @@ -1060,8 +1061,7 @@ public class TaskAttemptImpl implements TaskAttempt, TaskAttemptStartedEvent startEvt = new TaskAttemptStartedEvent( attemptId, getVertex().getName(), launchTime, containerId, containerNodeId, - inProgressLogsUrl, completedLogsUrl, nodeHttpAddress, creationTime, creationCausalTA, - allocationTime); + inProgressLogsUrl, completedLogsUrl, nodeHttpAddress); this.appContext.getHistoryHandler().handle( new DAGHistoryEvent(getDAGID(), startEvt)); } @@ -1073,7 +1073,7 @@ public class TaskAttemptImpl implements TaskAttempt, TaskAttemptFinishedEvent finishEvt = new TaskAttemptFinishedEvent( attemptId, getVertex().getName(), getLaunchTime(), getFinishTime(), TaskAttemptState.SUCCEEDED, null, - "", getCounters(), lastDataEvents); + "", getCounters(), lastDataEvents, creationTime, creationCausalTA, allocationTime); // FIXME how do we store information regd completion events this.appContext.getHistoryHandler().handle( new DAGHistoryEvent(getDAGID(), finishEvt)); @@ -1090,7 +1090,8 @@ public class TaskAttemptImpl implements TaskAttempt, finishTime, state, terminationCause, StringUtils.join( - getDiagnostics(), LINE_SEPARATOR), getCounters(), lastDataEvents); + getDiagnostics(), LINE_SEPARATOR), getCounters(), lastDataEvents, + creationTime, creationCausalTA, allocationTime); // FIXME how do we store information regd completion events this.appContext.getHistoryHandler().handle( new DAGHistoryEvent(getDAGID(), finishEvt)); http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java ---------------------------------------------------------------------- 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 fbde635..9f24151 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 @@ -45,8 +45,11 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { private TezTaskAttemptID taskAttemptId; private String vertexName; + private long creationTime; + private long allocationTime; private long startTime; private long finishTime; + private TezTaskAttemptID creationCausalTA; private TaskAttemptState state; private String diagnostics; private TezCounters tezCounters; @@ -60,9 +63,15 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { TaskAttemptState state, TaskAttemptTerminationCause error, String diagnostics, TezCounters counters, - List dataEvents) { + List dataEvents, + long creationTime, + TezTaskAttemptID creationCausalTA, + long allocationTime) { this.taskAttemptId = taId; this.vertexName = vertexName; + this.creationCausalTA = creationCausalTA; + this.creationTime = creationTime; + this.allocationTime = allocationTime; this.startTime = startTime; this.finishTime = finishTime; this.state = state; @@ -99,7 +108,13 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { TaskAttemptFinishedProto.newBuilder(); builder.setTaskAttemptId(taskAttemptId.toString()) .setState(state.ordinal()) + .setCreationTime(creationTime) + .setAllocationTime(allocationTime) + .setStartTime(startTime) .setFinishTime(finishTime); + if (creationCausalTA != null) { + builder.setCreationCausalTA(creationCausalTA.toString()); + } if (diagnostics != null) { builder.setDiagnostics(diagnostics); } @@ -119,8 +134,14 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { public void fromProto(TaskAttemptFinishedProto proto) { this.taskAttemptId = TezTaskAttemptID.fromString(proto.getTaskAttemptId()); - this.finishTime = proto.getFinishTime(); this.state = TaskAttemptState.values()[proto.getState()]; + this.creationTime = proto.getCreationTime(); + this.allocationTime = proto.getAllocationTime(); + this.startTime = proto.getStartTime(); + this.finishTime = proto.getFinishTime(); + if (proto.hasCreationCausalTA()) { + this.creationCausalTA = TezTaskAttemptID.fromString(proto.getCreationCausalTA()); + } if (proto.hasDiagnostics()) { this.diagnostics = proto.getDiagnostics(); } @@ -158,6 +179,8 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { public String toString() { return "vertexName=" + vertexName + ", taskAttemptId=" + taskAttemptId + + ", creationTime=" + creationTime + + ", allocationTime=" + allocationTime + ", startTime=" + startTime + ", finishTime=" + finishTime + ", timeTaken=" + (finishTime - startTime) @@ -198,5 +221,17 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { public long getStartTime() { return startTime; } + + public long getCreationTime() { + return creationTime; + } + + public long getAllocationTime() { + return allocationTime; + } + + public TezTaskAttemptID getCreationCausalTA() { + return creationCausalTA; + } } http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java ---------------------------------------------------------------------- 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 4d15fb9..a58b49e 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 @@ -40,15 +40,12 @@ public class TaskAttemptStartedEvent implements HistoryEvent { private ContainerId containerId; private NodeId nodeId; private String nodeHttpAddress; - private TezTaskAttemptID creationCausalTA; - private long creationTime; - private long allocationTime; public TaskAttemptStartedEvent(TezTaskAttemptID taId, String vertexName, long launchTime, ContainerId containerId, NodeId nodeId, String inProgressLogsUrl, String completedLogsUrl, - String nodeHttpAddress, long creationTime, TezTaskAttemptID creationCausalTA, long allocationTime) { + String nodeHttpAddress) { this.taskAttemptId = taId; this.vertexName = vertexName; this.launchTime = launchTime; @@ -57,9 +54,6 @@ public class TaskAttemptStartedEvent implements HistoryEvent { this.inProgressLogsUrl = inProgressLogsUrl; this.completedLogsUrl = completedLogsUrl; this.nodeHttpAddress = nodeHttpAddress; - this.creationTime = creationTime; - this.creationCausalTA = creationCausalTA; - this.allocationTime = allocationTime; } public TaskAttemptStartedEvent() { @@ -85,12 +79,7 @@ public class TaskAttemptStartedEvent implements HistoryEvent { builder.setTaskAttemptId(taskAttemptId.toString()) .setStartTime(launchTime) .setContainerId(containerId.toString()) - .setNodeId(nodeId.toString()) - .setCreationTime(creationTime) - .setAllocationTime(allocationTime); - if (creationCausalTA != null) { - builder.setCreationCausalTA(creationCausalTA.toString()); - } + .setNodeId(nodeId.toString()); return builder.build(); } @@ -99,11 +88,6 @@ public class TaskAttemptStartedEvent implements HistoryEvent { this.launchTime = proto.getStartTime(); this.containerId = ConverterUtils.toContainerId(proto.getContainerId()); this.nodeId = ConverterUtils.toNodeId(proto.getNodeId()); - this.creationTime = proto.getCreationTime(); - this.allocationTime = proto.getAllocationTime(); - if (proto.hasCreationCausalTA()) { - this.creationCausalTA = TezTaskAttemptID.fromString(proto.getCreationCausalTA()); - } } @Override @@ -124,8 +108,6 @@ public class TaskAttemptStartedEvent implements HistoryEvent { public String toString() { return "vertexName=" + vertexName + ", taskAttemptId=" + taskAttemptId - + ", creationTime=" + creationTime - + ", allocationTime=" + allocationTime + ", startTime=" + launchTime + ", containerId=" + containerId + ", nodeId=" + nodeId @@ -141,18 +123,6 @@ public class TaskAttemptStartedEvent implements HistoryEvent { return launchTime; } - public long getCreationTime() { - return creationTime; - } - - public long getAllocationTime() { - return allocationTime; - } - - public TezTaskAttemptID getCreationCausalTA() { - return creationCausalTA; - } - public ContainerId getContainerId() { return containerId; } http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java ---------------------------------------------------------------------- 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 411d677..649eb61 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 @@ -520,9 +520,14 @@ public class HistoryEventJsonConversion { jsonObject.put(ATSConstants.EVENTS, events); JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.CREATION_TIME, event.getCreationTime()); + otherInfo.put(ATSConstants.ALLOCATION_TIME, event.getAllocationTime()); otherInfo.put(ATSConstants.START_TIME, event.getStartTime()); otherInfo.put(ATSConstants.FINISH_TIME, event.getFinishTime()); otherInfo.put(ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime())); + if (event.getCreationCausalTA() != null) { + otherInfo.put(ATSConstants.CREATION_CAUSAL_ATTEMPT, event.getCreationCausalTA().toString()); + } otherInfo.put(ATSConstants.STATUS, event.getState().name()); if (event.getTaskAttemptError() != null) { otherInfo.put(ATSConstants.TASK_ATTEMPT_ERROR_ENUM, event.getTaskAttemptError().name()); @@ -577,11 +582,6 @@ public class HistoryEventJsonConversion { JSONObject otherInfo = new JSONObject(); otherInfo.put(ATSConstants.IN_PROGRESS_LOGS_URL, event.getInProgressLogsUrl()); otherInfo.put(ATSConstants.COMPLETED_LOGS_URL, event.getCompletedLogsUrl()); - otherInfo.put(ATSConstants.CREATION_TIME, event.getCreationTime()); - otherInfo.put(ATSConstants.ALLOCATION_TIME, event.getAllocationTime()); - if (event.getCreationCausalTA() != null) { - otherInfo.put(ATSConstants.CREATION_CAUSAL_ATTEMPT, event.getCreationCausalTA().toString()); - } jsonObject.put(ATSConstants.OTHER_INFO, otherInfo); return jsonObject; http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/main/proto/HistoryEvents.proto ---------------------------------------------------------------------- diff --git a/tez-dag/src/main/proto/HistoryEvents.proto b/tez-dag/src/main/proto/HistoryEvents.proto index 232f1b7..5392d8a 100644 --- a/tez-dag/src/main/proto/HistoryEvents.proto +++ b/tez-dag/src/main/proto/HistoryEvents.proto @@ -164,9 +164,6 @@ message TaskAttemptStartedProto { optional int64 start_time = 2; optional string container_id = 3; optional string node_id = 4; - optional int64 creation_time = 5; - optional string creation_causal_t_a = 6; - optional int64 allocation_time = 7; } message DataEventDependencyInfoProto { @@ -176,12 +173,16 @@ message DataEventDependencyInfoProto { message TaskAttemptFinishedProto { optional string task_attempt_id = 1; - optional int64 finish_time = 2; - optional int32 state = 3; - optional string diagnostics = 4; - optional TezCountersProto counters = 5; - optional string error_enum = 6; - repeated DataEventDependencyInfoProto data_events = 7; + optional int64 creation_time = 2; + optional int64 allocation_time = 3; + optional int64 start_time = 4; + optional int64 finish_time = 5; + optional string creation_causal_t_a = 6; + optional int32 state = 7; + optional string diagnostics = 8; + optional TezCountersProto counters = 9; + optional string error_enum = 10; + repeated DataEventDependencyInfoProto data_events = 11; } message EventMetaDataProto { http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java ---------------------------------------------------------------------- diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java index 53f1856..1aba5fa 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java @@ -157,14 +157,9 @@ public class TestTaskAttemptRecovery { } private void restoreFromTAStartEvent() { - TezTaskAttemptID causalId = TezTaskAttemptID.getInstance(taId.getTaskID(), taId.getId()+1); TaskAttemptState recoveredState = ta.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, - startTime, mock(ContainerId.class), mock(NodeId.class), "", "", "", creationTime, causalId, - allocationTime)); - assertEquals(causalId, ta.getCreationCausalAttempt()); - assertEquals(creationTime, ta.getCreationTime()); - assertEquals(allocationTime, ta.getAllocationTime()); + startTime, mock(ContainerId.class), mock(NodeId.class), "", "", "")); assertEquals(startTime, ta.getLaunchTime()); assertEquals(TaskAttemptState.RUNNING, recoveredState); } @@ -172,6 +167,7 @@ public class TestTaskAttemptRecovery { private void restoreFromTAFinishedEvent(TaskAttemptState state) { String diag = "test_diag"; TezCounters counters = mock(TezCounters.class); + TezTaskAttemptID causalId = TezTaskAttemptID.getInstance(taId.getTaskID(), taId.getId()+1); TaskAttemptTerminationCause errorEnum = null; if (state != TaskAttemptState.SUCCEEDED) { @@ -185,7 +181,11 @@ public class TestTaskAttemptRecovery { events.add(new DataEventDependencyInfo(lastDataEventTime, lastDataEventTA)); TaskAttemptState recoveredState = ta.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, - startTime, finishTime, state, errorEnum, diag, counters, events)); + startTime, finishTime, state, errorEnum, diag, counters, events, creationTime, + causalId, allocationTime)); + assertEquals(causalId, ta.getCreationCausalAttempt()); + assertEquals(creationTime, ta.getCreationTime()); + assertEquals(allocationTime, ta.getAllocationTime()); assertEquals(startTime, ta.getLaunchTime()); assertEquals(finishTime, ta.getFinishTime()); assertEquals(counters, ta.reportedStatus.counters); @@ -321,7 +321,7 @@ public class TestTaskAttemptRecovery { TaskAttemptState recoveredState = ta.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, startTime, finishTime, TaskAttemptState.KILLED, - TaskAttemptTerminationCause.APPLICATION_ERROR, "", new TezCounters(), null)); + TaskAttemptTerminationCause.APPLICATION_ERROR, "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskAttemptState.KILLED, recoveredState); } } http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java ---------------------------------------------------------------------- diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java index b6d4c10..bea423a 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java @@ -215,7 +215,7 @@ public class TestTaskRecovery { long taStartTime = taskStartTime + 100L; TaskState recoveredState = task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, - taStartTime, mock(ContainerId.class), mock(NodeId.class), "", "", "", 0, null, 0)); + taStartTime, mock(ContainerId.class), mock(NodeId.class), "", "", "")); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(0, task.getFinishedAttemptsCount()); assertEquals(taskScheduledTime, task.scheduledTime); @@ -286,7 +286,7 @@ public class TestTaskRecovery { restoreFromTaskStartEvent(); TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId()); task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, - 0L, 0L, TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT,"", new TezCounters(), null)); + 0L, 0L, TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT,"", new TezCounters(), null, 0, null, 0)); task.handle(new TaskEventRecoverTask(task.getTaskId())); // wait for the second task attempt is scheduled dispatcher.await(); @@ -307,7 +307,7 @@ public class TestTaskRecovery { restoreFromTaskStartEvent(); TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId()); task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, - 0L, 0L, TaskAttemptState.FAILED, TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED,"", new TezCounters(), null)); + 0L, 0L, TaskAttemptState.FAILED, TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED,"", new TezCounters(), null, 0, null, 0)); task.handle(new TaskEventRecoverTask(task.getTaskId())); // wait for the second task attempt is scheduled dispatcher.await(); @@ -329,7 +329,7 @@ public class TestTaskRecovery { TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId()); try { task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, - 0L, 0L, TaskAttemptState.SUCCEEDED, null ,"", new TezCounters(), null)); + 0L, 0L, TaskAttemptState.SUCCEEDED, null ,"", new TezCounters(), null, 0, null, 0)); fail("Should fail due to no TaskAttemptStartedEvent but with TaskAttemptFinishedEvent(Succeeded)"); } catch (TezUncheckedException e) { assertTrue(e.getMessage().contains("Could not find task attempt when trying to recover")); @@ -372,7 +372,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.SUCCEEDED, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -405,7 +405,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.FAILED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -438,7 +438,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.KILLED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -473,7 +473,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.SUCCEEDED, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -516,7 +516,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.SUCCEEDED, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -528,7 +528,7 @@ public class TestTaskRecovery { recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.FAILED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -563,7 +563,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.SUCCEEDED, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -575,7 +575,7 @@ public class TestTaskRecovery { recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.KILLED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -614,7 +614,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.SUCCEEDED, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -654,7 +654,7 @@ public class TestTaskRecovery { TaskState recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.SUCCEEDED, recoveredState); assertEquals(1, task.getAttempts().size()); assertEquals(1, task.getFinishedAttemptsCount()); @@ -721,7 +721,7 @@ public class TestTaskRecovery { TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId()); TaskState recoveredState = task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, - taStartTime, mock(ContainerId.class), mock(NodeId.class), "", "", "", 0, null, 0)); + taStartTime, mock(ContainerId.class), mock(NodeId.class), "", "", "")); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(TaskAttemptStateInternal.NEW, ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState()); @@ -735,7 +735,7 @@ public class TestTaskRecovery { recoveredState = task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, taStartTime, taFinishTime, TaskAttemptState.KILLED, null, - "", new TezCounters(), null)); + "", new TezCounters(), null, 0, null, 0)); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(TaskAttemptStateInternal.NEW, ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState()); @@ -774,9 +774,9 @@ public class TestTaskRecovery { for (int i = 0; i < maxFailedAttempts; ++i) { TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId()); task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L, - mock(ContainerId.class), mock(NodeId.class), "", "", "", 0, null, 0)); + mock(ContainerId.class), mock(NodeId.class), "", "", "")); task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0, - 0, TaskAttemptState.KILLED, null, "", null, null)); + 0, TaskAttemptState.KILLED, null, "", null, null, 0, null, 0)); } assertEquals(maxFailedAttempts, task.getAttempts().size()); assertEquals(0, task.failedAttempts); @@ -804,9 +804,9 @@ public class TestTaskRecovery { for (int i = 0; i < maxFailedAttempts; ++i) { TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId()); task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L, - mock(ContainerId.class), mock(NodeId.class), "", "", "", 0, null, 0)); + mock(ContainerId.class), mock(NodeId.class), "", "", "")); task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0, - 0, TaskAttemptState.FAILED, null, "", null, null)); + 0, TaskAttemptState.FAILED, null, "", null, null, 0, null, 0)); } assertEquals(maxFailedAttempts, task.getAttempts().size()); assertEquals(maxFailedAttempts, task.failedAttempts); @@ -834,9 +834,9 @@ public class TestTaskRecovery { for (int i = 0; i < maxFailedAttempts - 1; ++i) { TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId()); task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L, - mock(ContainerId.class), mock(NodeId.class), "", "", "", 0, null, 0)); + mock(ContainerId.class), mock(NodeId.class), "", "", "")); task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0, - 0, TaskAttemptState.FAILED, null, "", null, null)); + 0, TaskAttemptState.FAILED, null, "", null, null, 0, null, 0)); } assertEquals(maxFailedAttempts - 1, task.getAttempts().size()); assertEquals(maxFailedAttempts - 1, task.failedAttempts); @@ -844,7 +844,7 @@ public class TestTaskRecovery { TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(task.getTaskId()); TaskState recoveredState = task.restoreFromEvent(new TaskAttemptStartedEvent(newTaskAttemptId, - vertexName, 0, mock(ContainerId.class), mock(NodeId.class), "", "", "", 0, null, 0)); + vertexName, 0, mock(ContainerId.class), mock(NodeId.class), "", "", "")); assertEquals(TaskState.RUNNING, recoveredState); assertEquals(TaskAttemptStateInternal.NEW, http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java ---------------------------------------------------------------------- diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java index 5c8c90e..5a5cc53 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 @@ -480,9 +480,7 @@ public class TestHistoryEventsProtoConversion { "vertex1", 10009l, ContainerId.newInstance( ApplicationAttemptId.newInstance( ApplicationId.newInstance(0, 1), 1), 1001), NodeId.newInstance( - "host1", 19999), "inProgress", "Completed", "nodeHttpAddress", 1024, - TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance( - TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 0), 1024 + "host1", 19999), "inProgress", "Completed", "nodeHttpAddress" ); TaskAttemptStartedEvent deserializedEvent = (TaskAttemptStartedEvent) testProtoConversion(event); @@ -492,14 +490,6 @@ public class TestHistoryEventsProtoConversion { deserializedEvent.getContainerId()); Assert.assertEquals(event.getNodeId(), deserializedEvent.getNodeId()); - Assert.assertEquals(event.getStartTime(), - deserializedEvent.getStartTime()); - Assert.assertEquals(event.getCreationTime(), - deserializedEvent.getCreationTime()); - Assert.assertEquals(event.getAllocationTime(), - deserializedEvent.getAllocationTime()); - Assert.assertEquals(event.getCreationCausalTA(), - deserializedEvent.getCreationCausalTA()); logEvents(event, deserializedEvent); } @@ -509,13 +499,23 @@ public class TestHistoryEventsProtoConversion { TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance( TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 1), "vertex1", 10001l, 1000434444l, TaskAttemptState.FAILED, - null, null, null, null); + null, null, null, null, 2048, + TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance( + TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 0), 1024); TaskAttemptFinishedEvent deserializedEvent = (TaskAttemptFinishedEvent) testProtoConversion(event); Assert.assertEquals(event.getTaskAttemptID(), deserializedEvent.getTaskAttemptID()); + Assert.assertEquals(event.getCreationTime(), + deserializedEvent.getCreationTime()); + Assert.assertEquals(event.getAllocationTime(), + deserializedEvent.getAllocationTime()); + Assert.assertEquals(event.getStartTime(), + deserializedEvent.getStartTime()); Assert.assertEquals(event.getFinishTime(), deserializedEvent.getFinishTime()); + Assert.assertEquals(event.getCreationCausalTA(), + deserializedEvent.getCreationCausalTA()); Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics()); Assert.assertEquals(event.getState(), @@ -535,7 +535,7 @@ public class TestHistoryEventsProtoConversion { TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance( TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 1), "vertex1", 10001l, 1000434444l, TaskAttemptState.FAILED, - TaskAttemptTerminationCause.APPLICATION_ERROR, "diagnose", new TezCounters(), events); + TaskAttemptTerminationCause.APPLICATION_ERROR, "diagnose", new TezCounters(), events, 0, null, 0); TaskAttemptFinishedEvent deserializedEvent = (TaskAttemptFinishedEvent) testProtoConversion(event); Assert.assertEquals(event.getTaskAttemptID(), http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java ---------------------------------------------------------------------- diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java index 711e4bb..003a04f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java @@ -160,11 +160,11 @@ public class TestHistoryEventJsonConversion { break; case TASK_ATTEMPT_STARTED: event = new TaskAttemptStartedEvent(tezTaskAttemptID, "v1", random.nextInt(), containerId, - nodeId, null, null, "nodeHttpAddress", 0, null, 0); + nodeId, null, null, "nodeHttpAddress"); break; case TASK_ATTEMPT_FINISHED: event = new TaskAttemptFinishedEvent(tezTaskAttemptID, "v1", random.nextInt(), - random.nextInt(), TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT, null, null, null); + random.nextInt(), TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT, null, null, null, 0, null, 0); break; case CONTAINER_LAUNCHED: event = new ContainerLaunchedEvent(containerId, random.nextInt(), http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java ---------------------------------------------------------------------- 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 4685a61..c5ad816 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 @@ -426,7 +426,14 @@ public class HistoryEventTimelineConversion { atsEntity.addPrimaryFilter(ATSConstants.STATUS, event.getState().name()); + atsEntity.addOtherInfo(ATSConstants.CREATION_TIME, event.getCreationTime()); + atsEntity.addOtherInfo(ATSConstants.ALLOCATION_TIME, event.getAllocationTime()); + atsEntity.addOtherInfo(ATSConstants.START_TIME, event.getStartTime()); atsEntity.addOtherInfo(ATSConstants.FINISH_TIME, event.getFinishTime()); + if (event.getCreationCausalTA() != null) { + atsEntity.addOtherInfo(ATSConstants.CREATION_CAUSAL_ATTEMPT, + event.getCreationCausalTA().toString()); + } atsEntity.addOtherInfo(ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime())); atsEntity.addOtherInfo(ATSConstants.STATUS, event.getState().name()); if (event.getTaskAttemptError() != null) { @@ -473,12 +480,6 @@ public class HistoryEventTimelineConversion { atsEntity.addOtherInfo(ATSConstants.NODE_HTTP_ADDRESS, event.getNodeHttpAddress()); atsEntity.addOtherInfo(ATSConstants.CONTAINER_ID, event.getContainerId().toString()); atsEntity.addOtherInfo(ATSConstants.STATUS, TaskAttemptState.RUNNING.name()); - atsEntity.addOtherInfo(ATSConstants.CREATION_TIME, event.getCreationTime()); - atsEntity.addOtherInfo(ATSConstants.ALLOCATION_TIME, event.getAllocationTime()); - if (event.getCreationCausalTA() != null) { - atsEntity.addOtherInfo(ATSConstants.CREATION_CAUSAL_ATTEMPT, - event.getCreationCausalTA().toString()); - } return atsEntity; } http://git-wip-us.apache.org/repos/asf/tez/blob/a17a08d3/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java ---------------------------------------------------------------------- diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java index 2849c10..6e9f737 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java @@ -170,11 +170,11 @@ public class TestHistoryEventTimelineConversion { break; case TASK_ATTEMPT_STARTED: event = new TaskAttemptStartedEvent(tezTaskAttemptID, "v1", random.nextInt(), containerId, - nodeId, null, null, "nodeHttpAddress", 0, null, 0); + nodeId, null, null, "nodeHttpAddress"); break; case TASK_ATTEMPT_FINISHED: event = new TaskAttemptFinishedEvent(tezTaskAttemptID, "v1", random.nextInt(), - random.nextInt(), TaskAttemptState.FAILED, TaskAttemptTerminationCause.OUTPUT_LOST, null, null, null); + random.nextInt(), TaskAttemptState.FAILED, TaskAttemptTerminationCause.OUTPUT_LOST, null, null, null, 0, null, 0); break; case CONTAINER_LAUNCHED: event = new ContainerLaunchedEvent(containerId, random.nextInt(), @@ -453,8 +453,10 @@ public class TestHistoryEventTimelineConversion { @Test(timeout = 5000) public void testConvertTaskAttemptFinishedEvent() { String vertexName = "testVertex"; - long startTime = random.nextLong(); - long finishTime = startTime + 1234; + long creationTime = random.nextLong(); + long startTime = creationTime + 1000; + long allocationTime = creationTime + 1001; + long finishTime = startTime + 1002; TaskAttemptState state = TaskAttemptState .values()[random.nextInt(TaskAttemptState.values().length)]; TaskAttemptTerminationCause error = TaskAttemptTerminationCause @@ -467,7 +469,8 @@ public class TestHistoryEventTimelineConversion { events.add(new DataEventDependencyInfo(lastDataEventTime, tezTaskAttemptID)); TaskAttemptFinishedEvent event = new TaskAttemptFinishedEvent(tezTaskAttemptID, vertexName, - startTime, finishTime, state, error, diagnostics, counters, events); + startTime, finishTime, state, error, diagnostics, counters, events, creationTime, + tezTaskAttemptID, allocationTime); TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event); Assert.assertEquals(tezTaskAttemptID.toString(), timelineEntity.getEntityId()); Assert.assertEquals(EntityTypes.TEZ_TASK_ATTEMPT_ID.name(), timelineEntity.getEntityType()); @@ -490,7 +493,12 @@ public class TestHistoryEventTimelineConversion { Assert.assertEquals(finishTime, evt.getTimestamp()); final Map otherInfo = timelineEntity.getOtherInfo(); - Assert.assertEquals(7, otherInfo.size()); + Assert.assertEquals(11, otherInfo.size()); + Assert.assertEquals(tezTaskAttemptID.toString(), + timelineEntity.getOtherInfo().get(ATSConstants.CREATION_CAUSAL_ATTEMPT)); + Assert.assertEquals(creationTime, timelineEntity.getOtherInfo().get(ATSConstants.CREATION_TIME)); + Assert.assertEquals(allocationTime, timelineEntity.getOtherInfo().get(ATSConstants.ALLOCATION_TIME)); + Assert.assertEquals(startTime, timelineEntity.getOtherInfo().get(ATSConstants.START_TIME)); Assert.assertEquals(finishTime, otherInfo.get(ATSConstants.FINISH_TIME)); Assert.assertEquals(finishTime - startTime, otherInfo.get(ATSConstants.TIME_TAKEN)); Assert.assertEquals(state.name(), otherInfo.get(ATSConstants.STATUS)); @@ -744,11 +752,8 @@ public class TestHistoryEventTimelineConversion { @Test(timeout = 5000) public void testConvertTaskAttemptStartedEvent() { long startTime = random.nextLong(); - long creationTime = 1024; - long allocationTime = 1024; TaskAttemptStartedEvent event = new TaskAttemptStartedEvent(tezTaskAttemptID, "v1", - startTime, containerId, nodeId, "inProgressURL", "logsURL", "nodeHttpAddress", - creationTime, tezTaskAttemptID, allocationTime); + startTime, containerId, nodeId, "inProgressURL", "logsURL", "nodeHttpAddress"); TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event); Assert.assertEquals(EntityTypes.TEZ_TASK_ATTEMPT_ID.name(), timelineEntity.getEntityType()); @@ -793,10 +798,6 @@ public class TestHistoryEventTimelineConversion { timelineEntity.getOtherInfo().get(ATSConstants.NODE_HTTP_ADDRESS)); Assert.assertTrue(TaskAttemptState.RUNNING.name() .equals(timelineEntity.getOtherInfo().get(ATSConstants.STATUS))); - Assert.assertEquals(tezTaskAttemptID.toString(), - timelineEntity.getOtherInfo().get(ATSConstants.CREATION_CAUSAL_ATTEMPT)); - Assert.assertEquals(creationTime, timelineEntity.getOtherInfo().get(ATSConstants.CREATION_TIME)); - Assert.assertEquals(allocationTime, timelineEntity.getOtherInfo().get(ATSConstants.ALLOCATION_TIME)); } @Test(timeout = 5000)