tez-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hit...@apache.org
Subject [2/3] TEZ-799. Generate data to be used for recovery. (hitesh)
Date Fri, 07 Feb 2014 04:05:32 GMT
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java
----------------------------------------------------------------------
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 257508a..cc9c3ad 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
@@ -19,27 +19,46 @@
 package org.apache.tez.dag.history.events;
 
 import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.DAGFinished;
-import org.apache.tez.dag.history.avro.HistoryEventType;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.SummaryEvent;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
+import org.apache.tez.dag.history.utils.DAGUtils;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGFinishedProto;
+import org.apache.tez.dag.utils.ProtoUtils;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
 
-public class DAGFinishedEvent implements HistoryEvent {
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
-  private DAGFinished datum = new DAGFinished();
-  // FIXME remove this when we have a proper history
-  private final TezCounters tezCounters;
+public class DAGFinishedEvent implements HistoryEvent, SummaryEvent {
+
+  private TezDAGID dagID;
+  private long startTime;
+  private long finishTime;
+  private DAGStatus.State state;
+  private String diagnostics;
+  private TezCounters tezCounters;
+
+  public DAGFinishedEvent() {
+  }
 
   public DAGFinishedEvent(TezDAGID dagId, long startTime,
       long finishTime, DAGStatus.State state,
       String diagnostics, TezCounters counters) {
-    datum.dagId = dagId.toString();
-    datum.startTime = startTime;
-    datum.finishTime = finishTime;
-    datum.status = state.name();
-    datum.diagnostics = diagnostics;
-    tezCounters = counters;
+    this.dagID = dagId;
+    this.startTime = startTime;
+    this.finishTime = finishTime;
+    this.state = state;
+    this.diagnostics = diagnostics;
+    this.tezCounters = counters;
   }
 
   @Override
@@ -48,26 +67,96 @@ public class DAGFinishedEvent implements HistoryEvent {
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY,
+        dagID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE,
+        EntityTypes.TEZ_DAG_ID.name());
+
+    // Related Entities not needed as should have been done in
+    // dag submission event
+
+    // TODO decide whether this goes into different events,
+    // event info or other info.
+    JSONArray events = new JSONArray();
+    JSONObject finishEvent = new JSONObject();
+    finishEvent.put(ATSConstants.TIMESTAMP, finishTime);
+    finishEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.DAG_FINISHED.name());
+    events.put(finishEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.START_TIME, startTime);
+    otherInfo.put(ATSConstants.FINISH_TIME, finishTime);
+    otherInfo.put(ATSConstants.TIME_TAKEN, (finishTime - startTime));
+    otherInfo.put(ATSConstants.STATUS, state.name());
+    otherInfo.put(ATSConstants.DIAGNOSTICS, diagnostics);
+    otherInfo.put(ATSConstants.COUNTERS,
+        DAGUtils.convertCountersToJSON(this.tezCounters));
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public DAGFinishedProto toProto() {
+    return DAGFinishedProto.newBuilder()
+        .setDagId(dagID.toString())
+        .setState(state.ordinal())
+        .setDiagnostics(diagnostics)
+        .setFinishTime(finishTime)
+        .setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters))
+        .build();
+  }
+
+  public void fromProto(DAGFinishedProto proto) {
+    this.dagID = TezDAGID.fromString(proto.getDagId());
+    this.finishTime = proto.getFinishTime();
+    this.state = DAGStatus.State.values()[proto.getState()];
+    this.diagnostics = proto.getDiagnostics();
+    this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
+        proto.getCounters());
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (DAGFinished) blob;
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    DAGFinishedProto proto = DAGFinishedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "dagId=" + datum.dagId
-        + ", startTime=" + datum.startTime
-        + ", finishTime=" + datum.finishTime
-        + ", timeTaken=" + (datum.finishTime - datum.startTime)
-        + ", status=" + datum.status
-        + ", diagnostics=" + datum.diagnostics
+    return "dagId=" + dagID
+        + ", startTime=" + startTime
+        + ", finishTime=" + finishTime
+        + ", timeTaken=" + (finishTime - startTime)
+        + ", status=" + state.name()
+        + ", diagnostics=" + diagnostics
         + ", counters="
         + tezCounters.toString()
             .replaceAll("\\n", ", ").replaceAll("\\s+", " ");
   }
+
+  @Override
+  public void toSummaryProtoStream(OutputStream outputStream) throws IOException {
+    ProtoUtils.toSummaryEventProto(dagID, finishTime,
+        HistoryEventType.DAG_FINISHED).writeDelimitedTo(outputStream);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..20479e6
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.history.events;
+
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+// TODO fix class
+public class DAGInitializedEvent implements HistoryEvent {
+
+  private TezDAGID dagID;
+  private long initTime;
+
+  public DAGInitializedEvent() {
+  }
+
+  public DAGInitializedEvent(TezDAGID dagID, long initTime) {
+    this.dagID = dagID;
+    this.initTime = initTime;
+  }
+
+  @Override
+  public HistoryEventType getEventType() {
+    return HistoryEventType.DAG_INITIALIZED;
+  }
+
+  @Override
+  public JSONObject convertToATSJSON() throws JSONException {
+    // TODO
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "dagID=" + dagID
+        + ", initTime=" + initTime;
+  }
+
+  public RecoveryProtos.DAGInitializedProto toProto() {
+    return RecoveryProtos.DAGInitializedProto.newBuilder()
+        .setDagId(dagID.toString())
+        .setInitTime(initTime)
+        .build();
+  }
+
+  public void fromProto(RecoveryProtos.DAGInitializedProto proto) {
+    this.dagID = TezDAGID.fromString(proto.getDagId());
+    this.initTime = proto.getInitTime();
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    RecoveryProtos.DAGInitializedProto proto =
+        RecoveryProtos.DAGInitializedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java
----------------------------------------------------------------------
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 719538c..4574753 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
@@ -19,19 +19,30 @@
 package org.apache.tez.dag.history.events;
 
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.DAGStarted;
-import org.apache.tez.dag.history.avro.HistoryEventType;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGStartedProto;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 public class DAGStartedEvent implements HistoryEvent {
 
-  private DAGStarted datum = new DAGStarted();
+  private TezDAGID dagID;
+  private long startTime;
+
+  public DAGStartedEvent() {
+  }
 
-  public DAGStartedEvent(TezDAGID dagId, long initTime,
-      long startTime) {
-    datum.dagId = dagId.toString();
-    datum.initTime = initTime;
-    datum.startTime = startTime;
+  public DAGStartedEvent(TezDAGID dagID, long startTime) {
+    this.dagID = dagID;
+    this.startTime = startTime;
   }
 
   @Override
@@ -40,20 +51,66 @@ public class DAGStartedEvent implements HistoryEvent {
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY,
+        dagID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE,
+        EntityTypes.TEZ_DAG_ID.name());
+
+    // Related Entities not needed as should have been done in
+    // dag submission event
+
+    // TODO decide whether this goes into different events,
+    // event info or other info.
+    JSONArray events = new JSONArray();
+    JSONObject startEvent = new JSONObject();
+    startEvent.put(ATSConstants.TIMESTAMP, startTime);
+    startEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.DAG_STARTED.name());
+    events.put(startEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    return jsonObject;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public DAGStartedProto toProto() {
+    return DAGStartedProto.newBuilder()
+        .setDagId(dagID.toString())
+        .setStartTime(startTime)
+        .build();
+  }
+
+  public void fromProto(DAGStartedProto proto) {
+    this.dagID = TezDAGID.fromString(proto.getDagId());
+    this.startTime = proto.getStartTime();
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (DAGStarted) blob;
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    DAGStartedProto proto = DAGStartedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "dagId=" + datum.dagId
-        + ", initTime=" + datum.initTime
-        + ", startTime=" + datum.startTime;
+    return "dagID=" + dagID
+        + ", startTime=" + startTime;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..3d24105
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.history.events;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tez.dag.api.records.DAGProtos;
+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.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
+import org.apache.tez.dag.history.utils.DAGUtils;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGSubmittedProto;
+import org.apache.tez.dag.utils.ProtoUtils;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+      public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
+
+  private TezDAGID dagID;
+  private long submitTime;
+  private DAGProtos.DAGPlan dagPlan;
+  private ApplicationAttemptId applicationAttemptId;
+
+  public DAGSubmittedEvent() {
+  }
+
+  public DAGSubmittedEvent(TezDAGID dagID, long submitTime,
+      DAGProtos.DAGPlan dagPlan, ApplicationAttemptId applicationAttemptId) {
+    this.dagID = dagID;
+    this.submitTime = submitTime;
+    this.dagPlan = dagPlan;
+    this.applicationAttemptId = applicationAttemptId;
+  }
+
+  @Override
+  public HistoryEventType getEventType() {
+    return HistoryEventType.DAG_SUBMITTED;
+  }
+
+  @Override
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY,
+        dagID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE,
+        EntityTypes.TEZ_DAG_ID.name());
+
+    // Related Entities
+    JSONArray relatedEntities = new JSONArray();
+    JSONObject tezAppEntity = new JSONObject();
+    tezAppEntity.put(ATSConstants.ENTITY,
+        "tez_" + applicationAttemptId.toString());
+    tezAppEntity.put(ATSConstants.ENTITY_TYPE,
+        EntityTypes.TEZ_APPLICATION_ATTEMPT.name());
+    JSONObject appEntity = new JSONObject();
+    appEntity.put(ATSConstants.ENTITY,
+        applicationAttemptId.getApplicationId().toString());
+    appEntity.put(ATSConstants.ENTITY_TYPE,
+        ATSConstants.APPLICATION_ID);
+    JSONObject appAttemptEntity = new JSONObject();
+    appAttemptEntity.put(ATSConstants.ENTITY,
+        applicationAttemptId.toString());
+    appAttemptEntity.put(ATSConstants.ENTITY_TYPE,
+        ATSConstants.APPLICATION_ATTEMPT_ID);
+
+    relatedEntities.put(tezAppEntity);
+    relatedEntities.put(appEntity);
+    relatedEntities.put(appAttemptEntity);
+    jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities);
+
+    // filters
+    JSONObject primaryFilters = new JSONObject();
+    primaryFilters.put(ATSConstants.DAG_NAME,
+        dagPlan.getName());
+    jsonObject.put(ATSConstants.PRIMARY_FILTERS, primaryFilters);
+
+    // TODO decide whether this goes into different events,
+    // event info or other info.
+    JSONArray events = new JSONArray();
+    JSONObject submitEvent = new JSONObject();
+    submitEvent.put(ATSConstants.TIMESTAMP, submitTime);
+    submitEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.DAG_SUBMITTED.name());
+    events.put(submitEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    // Other info such as dag plan
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.DAG_PLAN,
+        DAGUtils.generateSimpleJSONPlan(dagPlan));
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public DAGSubmittedProto toProto() {
+    return DAGSubmittedProto.newBuilder()
+        .setDagId(dagID.toString())
+        .setApplicationAttemptId(applicationAttemptId.toString())
+        .setDagPlan(dagPlan)
+        .setSubmitTime(submitTime)
+        .build();
+
+  }
+
+  public void fromProto(DAGSubmittedProto proto) {
+    this.dagID = TezDAGID.fromString(proto.getDagId());
+    this.dagPlan = proto.getDagPlan();
+    this.submitTime = proto.getSubmitTime();
+    this.applicationAttemptId = ConverterUtils.toApplicationAttemptId(
+        proto.getApplicationAttemptId());
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    DAGSubmittedProto proto = DAGSubmittedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
+  }
+
+  @Override
+  public String toString() {
+    return "dagID=" + dagID
+        + ", submitTime=" + submitTime;
+  }
+
+  @Override
+  public void toSummaryProtoStream(OutputStream outputStream) throws IOException {
+    ProtoUtils.toSummaryEventProto(dagID, submitTime,
+        HistoryEventType.DAG_SUBMITTED).writeDelimitedTo(outputStream);
+  }
+
+  public String getDAGName() {
+    if (dagPlan != null && dagPlan.hasName()) {
+      return dagPlan.getName();
+    }
+    return null;
+  }
+
+  public DAGProtos.DAGPlan getDAGPlan() {
+    return this.dagPlan;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/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 edd8f8b..97f3be3 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
@@ -19,17 +19,32 @@
 package org.apache.tez.dag.history.events;
 
 import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.HistoryEventType;
-import org.apache.tez.dag.history.avro.TaskAttemptFinished;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
+import org.apache.tez.dag.history.utils.DAGUtils;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskAttemptFinishedProto;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 public class TaskAttemptFinishedEvent implements HistoryEvent {
 
-  private TaskAttemptFinished datum = new TaskAttemptFinished();
-  // FIXME remove this when we have a proper history
-  private final TezCounters tezCounters;
+  private TezTaskAttemptID taskAttemptId;
+  private String vertexName;
+  private long startTime;
+  private long finishTime;
+  private TaskAttemptState state;
+  private String diagnostics;
+  private TezCounters tezCounters;
 
   public TaskAttemptFinishedEvent(TezTaskAttemptID taId,
       String vertexName,
@@ -38,42 +53,105 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
       TaskAttemptState state,
       String diagnostics,
       TezCounters counters) {
-    datum.taskAttemptId = taId.toString();
-    datum.vertexName = vertexName;
-    datum.startTime = startTime;
-    datum.finishTime = finishTime;
-    datum.status = state.name();
-    datum.diagnostics = diagnostics;
+    this.taskAttemptId = taId;
+    this.vertexName = vertexName;
+    this.startTime = startTime;
+    this.finishTime = finishTime;
+    this.state = state;
+    this.diagnostics = diagnostics;
     tezCounters = counters;
   }
 
+  public TaskAttemptFinishedEvent() {
+  }
+
   @Override
   public HistoryEventType getEventType() {
     return HistoryEventType.TASK_ATTEMPT_FINISHED;
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, taskAttemptId.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE,
+        EntityTypes.TEZ_TASK_ATTEMPT_ID.name());
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject finishEvent = new JSONObject();
+    finishEvent.put(ATSConstants.TIMESTAMP, finishTime);
+    finishEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.TASK_ATTEMPT_FINISHED.name());
+    events.put(finishEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.START_TIME, startTime);
+    otherInfo.put(ATSConstants.FINISH_TIME, finishTime);
+    otherInfo.put(ATSConstants.TIME_TAKEN, (finishTime - startTime));
+    otherInfo.put(ATSConstants.STATUS, state.name());
+    otherInfo.put(ATSConstants.DIAGNOSTICS, diagnostics);
+    otherInfo.put(ATSConstants.COUNTERS,
+        DAGUtils.convertCountersToJSON(this.tezCounters));
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public TaskAttemptFinishedProto toProto() {
+    return TaskAttemptFinishedProto.newBuilder()
+        .setTaskAttemptId(taskAttemptId.toString())
+        .setState(state.ordinal())
+        .setDiagnostics(diagnostics)
+        .setFinishTime(finishTime)
+        .setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters))
+        .build();
+  }
+
+  public void fromProto(TaskAttemptFinishedProto proto) {
+    this.taskAttemptId = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
+    this.finishTime = proto.getFinishTime();
+    this.state = TaskAttemptState.values()[proto.getState()];
+    this.diagnostics = proto.getDiagnostics();
+    this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
+        proto.getCounters());
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (TaskAttemptFinished) blob;
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    TaskAttemptFinishedProto proto =
+        TaskAttemptFinishedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "vertexName=" + datum.vertexName
-        + ", taskAttemptId=" + datum.taskAttemptId
-        + ", startTime=" + datum.startTime
-        + ", finishTime=" + datum.finishTime
-        + ", timeTaken=" + (datum.finishTime - datum.startTime)
-        + ", status=" + datum.status
-        + ", diagnostics=" + datum.diagnostics
+    return "vertexName=" + vertexName
+        + ", taskAttemptId=" + taskAttemptId
+        + ", startTime=" + startTime
+        + ", finishTime=" + finishTime
+        + ", timeTaken=" + (finishTime - startTime)
+        + ", status=" + state.name()
+        + ", diagnostics=" + diagnostics
         + ", counters="
         + tezCounters.toString()
             .replaceAll("\\n", ", ").replaceAll("\\s+", " ");
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/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 5d29b21..11a1c62 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
@@ -20,54 +20,142 @@ package org.apache.tez.dag.history.events;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.HistoryEventType;
-import org.apache.tez.dag.history.avro.TaskAttemptStarted;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskAttemptStartedProto;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 public class TaskAttemptStartedEvent implements HistoryEvent {
 
-  private TaskAttemptStarted datum = new TaskAttemptStarted();
-  private final String inProgressLogsUrl;
-  private final String completedLogsUrl;
+  private TezTaskAttemptID taskAttemptId;
+  private String inProgressLogsUrl;
+  private String completedLogsUrl;
+  private String vertexName;
+  private long startTime;
+  private ContainerId containerId;
+  private NodeId nodeId;
 
   public TaskAttemptStartedEvent(TezTaskAttemptID taId,
       String vertexName, long startTime,
       ContainerId containerId, NodeId nodeId,
       String inProgressLogsUrl, String completedLogsUrl) {
-    datum.taskAttemptId = taId.toString();
-    datum.vertexName = vertexName;
-    datum.startTime = startTime;
-    datum.containerId = containerId.toString();
-    datum.nodeId = nodeId.toString();
+    this.taskAttemptId = taId;
+    this.vertexName = vertexName;
+    this.startTime = startTime;
+    this.containerId = containerId;
+    this.nodeId = nodeId;
     this.inProgressLogsUrl = inProgressLogsUrl;
     this.completedLogsUrl = completedLogsUrl;
   }
 
+  public TaskAttemptStartedEvent() {
+  }
+
   @Override
   public HistoryEventType getEventType() {
     return HistoryEventType.TASK_ATTEMPT_STARTED;
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, taskAttemptId.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE,
+        EntityTypes.TEZ_TASK_ATTEMPT_ID.name());
+
+    // Related entities
+    JSONArray relatedEntities = new JSONArray();
+    JSONObject nodeEntity = new JSONObject();
+    nodeEntity.put(ATSConstants.ENTITY, nodeId.toString());
+    nodeEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.NODE_ID);
+
+    JSONObject containerEntity = new JSONObject();
+    containerEntity.put(ATSConstants.ENTITY, containerId.toString());
+    containerEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.CONTAINER_ID);
+
+    JSONObject taskEntity = new JSONObject();
+    taskEntity.put(ATSConstants.ENTITY, taskAttemptId.getTaskID().toString());
+    taskEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name());
+
+    relatedEntities.put(nodeEntity);
+    relatedEntities.put(containerEntity);
+    relatedEntities.put(taskEntity);
+    jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities);
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject startEvent = new JSONObject();
+    startEvent.put(ATSConstants.TIMESTAMP, startTime);
+    startEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.TASK_ATTEMPT_STARTED.name());
+    events.put(startEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    // Other info
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.IN_PROGRESS_LOGS_URL, inProgressLogsUrl);
+    otherInfo.put(ATSConstants.COMPLETED_LOGS_URL, completedLogsUrl);
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return false;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public TaskAttemptStartedProto toProto() {
+    return TaskAttemptStartedProto.newBuilder()
+        .setTaskAttemptId(taskAttemptId.toString())
+        .setStartTime(startTime)
+        .setContainerId(containerId.toString())
+        .setNodeId(nodeId.toString())
+        .build();
+  }
+
+  public void fromProto(TaskAttemptStartedProto proto) {
+    this.taskAttemptId = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
+    this.startTime = proto.getStartTime();
+    this.containerId = ConverterUtils.toContainerId(proto.getContainerId());
+    this.nodeId = ConverterUtils.toNodeId(proto.getNodeId());
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (TaskAttemptStarted) blob;
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    TaskAttemptStartedProto proto = TaskAttemptStartedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "vertexName=" + datum.vertexName
-        + ", taskAttemptId=" + datum.taskAttemptId
-        + ", startTime=" + datum.startTime
-        + ", containerId=" + datum.containerId
-        + ", nodeId=" + datum.nodeId
+    return "vertexName=" + vertexName
+        + ", taskAttemptId=" + taskAttemptId
+        + ", startTime=" + startTime
+        + ", containerId=" + containerId
+        + ", nodeId=" + nodeId
         + ", inProgressLogs=" + inProgressLogsUrl
         + ", completedLogs=" + completedLogsUrl;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
index 6eca970..dac2f9a 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
@@ -19,27 +19,44 @@
 package org.apache.tez.dag.history.events;
 
 import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.HistoryEventType;
-import org.apache.tez.dag.history.avro.TaskFinished;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
+import org.apache.tez.dag.history.utils.DAGUtils;
 import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskFinishedProto;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 public class TaskFinishedEvent implements HistoryEvent {
 
-  private TaskFinished datum = new TaskFinished();
-  // FIXME remove this when we have a proper history
-  private final TezCounters tezCounters;
+  private TezTaskID taskID;
+  private String vertexName;
+  private long startTime;
+  private long finishTime;
+  private TaskState state;
+  private TezCounters tezCounters;
 
-  public TaskFinishedEvent(TezTaskID taskId,
+  public TaskFinishedEvent(TezTaskID taskID,
       String vertexName, long startTime, long finishTime,
       TaskState state, TezCounters counters) {
-    datum.vertexName = vertexName;
-    datum.taskId = taskId.toString();
-    datum.startTime = startTime;
-    datum.finishTime = finishTime;
-    datum.status = state.name();
-    tezCounters = counters;
+    this.vertexName = vertexName;
+    this.taskID = taskID;
+    this.startTime = startTime;
+    this.finishTime = finishTime;
+    this.state = state;
+    this.tezCounters = counters;
+  }
+
+  public TaskFinishedEvent() {
   }
 
   @Override
@@ -48,26 +65,82 @@ public class TaskFinishedEvent implements HistoryEvent {
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, taskID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name());
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject finishEvent = new JSONObject();
+    finishEvent.put(ATSConstants.TIMESTAMP, finishTime);
+    finishEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.TASK_FINISHED.name());
+    events.put(finishEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.START_TIME, startTime);
+    otherInfo.put(ATSConstants.FINISH_TIME, finishTime);
+    otherInfo.put(ATSConstants.TIME_TAKEN, (finishTime - startTime));
+    otherInfo.put(ATSConstants.STATUS, state.name());
+    otherInfo.put(ATSConstants.COUNTERS,
+        DAGUtils.convertCountersToJSON(this.tezCounters));
+
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (TaskFinished) blob;
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public TaskFinishedProto toProto() {
+    return TaskFinishedProto.newBuilder()
+        .setTaskId(taskID.toString())
+        .setState(state.ordinal())
+        .setFinishTime(finishTime)
+        .setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters))
+        .build();
+  }
+
+  public void fromProto(TaskFinishedProto proto) {
+    this.taskID = TezTaskID.fromString(proto.getTaskId());
+    this.finishTime = proto.getFinishTime();
+    this.state = TaskState.values()[proto.getState()];
+    this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
+        proto.getCounters());
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    TaskFinishedProto proto = TaskFinishedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "vertexName=" + datum.vertexName
-        + ", taskId=" + datum.taskId
-        + ", startTime=" + datum.startTime
-        + ", finishTime=" + datum.finishTime
-        + ", timeTaken=" + (datum.finishTime - datum.startTime)
-        + ", status=" + datum.status
+    return "vertexName=" + vertexName
+        + ", taskId=" + taskID
+        + ", startTime=" + startTime
+        + ", finishTime=" + finishTime
+        + ", timeTaken=" + (finishTime - startTime)
+        + ", status=" + state.name()
         + ", counters="
         + tezCounters.toString()
             .replaceAll("\\n", ", ").replaceAll("\\s+", " ");
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java
----------------------------------------------------------------------
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 964cc79..9efa357 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
@@ -19,20 +19,35 @@
 package org.apache.tez.dag.history.events;
 
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.HistoryEventType;
-import org.apache.tez.dag.history.avro.TaskStarted;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
 import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskStartedProto;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 public class TaskStartedEvent implements HistoryEvent {
 
-  private TaskStarted datum = new TaskStarted();
+  private TezTaskID taskID;
+  private String vertexName;
+  private long scheduledTime;
+  private long startTime;
 
   public TaskStartedEvent(TezTaskID taskId,
-      String vertexName, long scheduledTime, long launchTime) {
-    datum.vertexName = vertexName;
-    datum.taskId = taskId.toString();
-    datum.scheduledTime = scheduledTime;
-    datum.launchTime = launchTime;
+      String vertexName, long scheduledTime, long startTime) {
+    this.vertexName = vertexName;
+    this.taskID = taskId;
+    this.scheduledTime = scheduledTime;
+    this.startTime = startTime;
+  }
+
+  public TaskStartedEvent() {
   }
 
   @Override
@@ -41,21 +56,80 @@ public class TaskStartedEvent implements HistoryEvent {
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, taskID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name());
+
+    // Related entities
+    JSONArray relatedEntities = new JSONArray();
+    JSONObject vertexEntity = new JSONObject();
+    vertexEntity.put(ATSConstants.ENTITY, taskID.getVertexID().toString());
+    vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name());
+    relatedEntities.put(vertexEntity);
+    jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities);
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject startEvent = new JSONObject();
+    startEvent.put(ATSConstants.TIMESTAMP, startTime);
+    startEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.TASK_STARTED.name());
+    events.put(startEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+    
+    // Other info
+    // TODO fix schedule/launch time to be events
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.START_TIME, startTime);
+    otherInfo.put(ATSConstants.SCHEDULED_TIME, scheduledTime);
+
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (TaskStarted) blob;
+  public boolean isRecoveryEvent() {
+    return false;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public TaskStartedProto toProto() {
+    return TaskStartedProto.newBuilder()
+        .setTaskId(taskID.toString())
+        .setLaunchTime(startTime)
+        .setScheduledTime(scheduledTime)
+        .build();
+  }
+
+  public void fromProto(TaskStartedProto proto) {
+    this.taskID = TezTaskID.fromString(proto.getTaskId());
+    this.startTime = proto.getLaunchTime();
+    this.scheduledTime = proto.getScheduledTime();
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    TaskStartedProto proto = TaskStartedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "vertexName=" + datum.vertexName
-        + ", taskId=" + datum.taskId
-        + ", scheduledTime=" + datum.scheduledTime
-        + ", launchTime=" + datum.launchTime;
+    return "vertexName=" + vertexName
+        + ", taskId=" + taskID.toString()
+        + ", scheduledTime=" + scheduledTime
+        + ", launchTime=" + startTime;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
new file mode 100644
index 0000000..7c2a16f
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.history.events;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.common.ProtoConverters;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TezDataMovementEventProto;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexDataMovementEventsGeneratedProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.RootInputDataInformationEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventType;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.EnumSet;
+import java.util.List;
+
+public class VertexDataMovementEventsGeneratedEvent implements HistoryEvent {
+
+  private static final Log LOG = LogFactory.getLog(
+      VertexDataMovementEventsGeneratedEvent.class);
+  private List<TezEvent> events;
+  private TezVertexID vertexID;
+
+  public VertexDataMovementEventsGeneratedEvent(TezVertexID vertexID,
+      List<TezEvent> events) {
+    this.vertexID = vertexID;
+    this.events = Lists.newArrayListWithCapacity(events.size());
+    for (TezEvent event : events) {
+      if (EnumSet.of(EventType.DATA_MOVEMENT_EVENT,
+          EventType.COMPOSITE_DATA_MOVEMENT_EVENT,
+          EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)
+              .contains(event.getEventType())) {
+        this.events.add(event);
+      }
+    }
+    if (events.isEmpty()) {
+      throw new RuntimeException("Invalid creation of VertexDataMovementEventsGeneratedEvent"
+        + ", no data movement/information events provided");
+    }
+  }
+
+  public VertexDataMovementEventsGeneratedEvent() {
+  }
+
+  @Override
+  public HistoryEventType getEventType() {
+    return HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED;
+  }
+
+  @Override
+  public JSONObject convertToATSJSON() throws JSONException {
+    return null;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return false;
+  }
+
+  static RecoveryProtos.EventMetaDataProto convertEventMetaDataToProto(
+      EventMetaData eventMetaData) {
+    RecoveryProtos.EventMetaDataProto.Builder builder =
+        RecoveryProtos.EventMetaDataProto.newBuilder()
+        .setProducerConsumerType(eventMetaData.getEventGenerator().ordinal())
+        .setEdgeVertexName(eventMetaData.getEdgeVertexName())
+        .setTaskVertexName(eventMetaData.getTaskVertexName());
+    if (eventMetaData.getTaskAttemptID() != null) {
+        builder.setTaskAttemptId(eventMetaData.getTaskAttemptID().toString());
+    }
+    return builder.build();
+  }
+
+  static EventMetaData convertEventMetaDataFromProto(
+      RecoveryProtos.EventMetaDataProto proto) {
+    TezTaskAttemptID attemptID = null;
+    if (proto.hasTaskAttemptId()) {
+      attemptID = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
+    }
+    return new EventMetaData(
+        EventMetaData.EventProducerConsumerType.values()[proto.getProducerConsumerType()],
+        proto.getTaskVertexName(),
+        proto.getEdgeVertexName(),
+        attemptID);
+  }
+
+  public VertexDataMovementEventsGeneratedProto toProto() {
+    List<TezDataMovementEventProto> tezEventProtos = null;
+    if (events != null) {
+      tezEventProtos = Lists.newArrayListWithCapacity(events.size());
+      for (TezEvent event : events) {
+        TezDataMovementEventProto.Builder evtBuilder =
+            TezDataMovementEventProto.newBuilder();
+        if (event.getEventType().equals(EventType.COMPOSITE_DATA_MOVEMENT_EVENT)) {
+          evtBuilder.setCompositeDataMovementEvent(
+              ProtoConverters.convertCompositeDataMovementEventToProto(
+                  (CompositeDataMovementEvent) event.getEvent()));
+        } else if (event.getEventType().equals(EventType.DATA_MOVEMENT_EVENT)) {
+          evtBuilder.setDataMovementEvent(
+              ProtoConverters.convertDataMovementEventToProto(
+                  (DataMovementEvent) event.getEvent()));
+        } else if (event.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
+          evtBuilder.setRootInputDataInformationEvent(
+              ProtoConverters.convertRootInputDataInformationEventToProto(
+                  (RootInputDataInformationEvent) event.getEvent()));
+        }
+        if (event.getSourceInfo() != null) {
+          evtBuilder.setSourceInfo(convertEventMetaDataToProto(event.getSourceInfo()));
+        }
+        if (event.getDestinationInfo() != null) {
+          evtBuilder.setDestinationInfo(convertEventMetaDataToProto(event.getDestinationInfo()));
+        }
+        tezEventProtos.add(evtBuilder.build());
+      }
+    }
+    return VertexDataMovementEventsGeneratedProto.newBuilder()
+        .setVertexId(vertexID.toString())
+        .addAllTezDataMovementEvent(tezEventProtos)
+        .build();
+  }
+
+  public void fromProto(VertexDataMovementEventsGeneratedProto proto) {
+    this.vertexID = TezVertexID.fromString(proto.getVertexId());
+    int eventCount = proto.getTezDataMovementEventCount();
+    if (eventCount > 0) {
+      this.events = Lists.newArrayListWithCapacity(eventCount);
+    }
+    for (TezDataMovementEventProto eventProto :
+        proto.getTezDataMovementEventList()) {
+      Event evt = null;
+      if (eventProto.hasCompositeDataMovementEvent()) {
+        evt = ProtoConverters.convertCompositeDataMovementEventFromProto(
+            eventProto.getCompositeDataMovementEvent());
+      } else if (eventProto.hasDataMovementEvent()) {
+        evt = ProtoConverters.convertDataMovementEventFromProto(
+            eventProto.getDataMovementEvent());
+      } else if (eventProto.hasRootInputDataInformationEvent()) {
+        evt = ProtoConverters.convertRootInputDataInformationEventFromProto(
+            eventProto.getRootInputDataInformationEvent());
+      }
+      EventMetaData sourceInfo = null;
+      EventMetaData destinationInfo = null;
+      if (eventProto.hasSourceInfo()) {
+        sourceInfo = convertEventMetaDataFromProto(eventProto.getSourceInfo());
+      }
+      if (eventProto.hasDestinationInfo()) {
+        destinationInfo = convertEventMetaDataFromProto(eventProto.getDestinationInfo());
+      }
+      TezEvent tezEvent = new TezEvent(evt, sourceInfo);
+      tezEvent.setDestinationInfo(destinationInfo);
+      this.events.add(tezEvent);
+    }
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    VertexDataMovementEventsGeneratedProto proto =
+        VertexDataMovementEventsGeneratedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
+  }
+
+  @Override
+  public String toString() {
+    return "vertexId=" + vertexID.toString()
+        + ", eventCount=" + (events != null ? events.size() : "null");
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
index f09150f..0f2b8a1 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
@@ -19,64 +19,144 @@
 package org.apache.tez.dag.history.events;
 
 import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.client.VertexStatus;
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.HistoryEventType;
-import org.apache.tez.dag.history.avro.VertexFinished;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
+import org.apache.tez.dag.history.utils.DAGUtils;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishedProto;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 public class VertexFinishedEvent implements HistoryEvent {
 
-  private VertexFinished datum = new VertexFinished();
-  // FIXME remove this when we have a proper history
-  private final TezCounters tezCounters;
+  private TezVertexID vertexID;
+  private String vertexName;
+  private long initRequestedTime;
+  private long initedTime;
+  private long startRequestedTime;
+  private long startTime;
+  private long finishTime;
+  private VertexStatus.State state;
+  private String diagnostics;
+  private TezCounters tezCounters;
 
   public VertexFinishedEvent(TezVertexID vertexId,
       String vertexName, long initRequestedTime, long initedTime, long startRequestedTime, long startedTime, long finishTime,
       VertexStatus.State state, String diagnostics,
       TezCounters counters) {
-    datum.vertexName = vertexName;
-    datum.vertexId = vertexId.toString();
-    datum.initRequestedTime = initRequestedTime;
-    datum.initedTime = initedTime;
-    datum.startRequestedTime = startRequestedTime;
-    datum.startedTime = startedTime;
-    datum.finishTime = finishTime;
-    datum.status = state.name();
-    datum.diagnostics = diagnostics;
+    this.vertexName = vertexName;
+    this.vertexID = vertexId;
+    this.initRequestedTime = initRequestedTime;
+    this.initedTime = initedTime;
+    this.startRequestedTime = startRequestedTime;
+    this.startTime = startedTime;
+    this.finishTime = finishTime;
+    this.state = state;
+    this.diagnostics = diagnostics;
     tezCounters = counters;
   }
 
+  public VertexFinishedEvent() {
+  }
+
   @Override
   public HistoryEventType getEventType() {
     return HistoryEventType.VERTEX_FINISHED;
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, vertexID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name());
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject finishEvent = new JSONObject();
+    finishEvent.put(ATSConstants.TIMESTAMP, finishTime);
+    finishEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.VERTEX_FINISHED.name());
+    events.put(finishEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.FINISH_TIME, finishTime);
+    otherInfo.put(ATSConstants.TIME_TAKEN, (finishTime - startTime));
+    otherInfo.put(ATSConstants.STATUS, state.name());
+    otherInfo.put(ATSConstants.DIAGNOSTICS, diagnostics);
+    otherInfo.put(ATSConstants.COUNTERS,
+        DAGUtils.convertCountersToJSON(this.tezCounters));
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public VertexFinishedProto toProto() {
+    return VertexFinishedProto.newBuilder()
+        .setVertexName(vertexName)
+        .setVertexId(vertexID.toString())
+        .setState(state.ordinal())
+        .setDiagnostics(diagnostics)
+        .setFinishTime(finishTime)
+        .setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters))
+        .build();
+  }
+
+  public void fromProto(VertexFinishedProto proto) {
+    this.vertexName = proto.getVertexName();
+    this.vertexID = TezVertexID.fromString(proto.getVertexId());
+    this.finishTime = proto.getFinishTime();
+    this.state = VertexStatus.State.values()[proto.getState()];
+    this.diagnostics = proto.getDiagnostics();
+    this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
+        proto.getCounters());
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (VertexFinished) blob;
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    VertexFinishedProto proto = VertexFinishedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "vertexName=" + datum.vertexName
-        + ", vertexId=" + datum.vertexId
-        + ", initRequestedTime=" + datum.initRequestedTime
-        + ", initedTime=" + datum.initedTime
-        + ", startRequestedTime=" + datum.startRequestedTime
-        + ", startedTime=" + datum.startedTime
-        + ", finishTime=" + datum.finishTime
-        + ", timeTaken=" + (datum.finishTime - datum.startedTime)
-        + ", status=" + datum.status
-        + ", diagnostics=" + datum.diagnostics
+    return "vertexName=" + vertexName
+        + ", vertexId=" + vertexID
+        + ", initRequestedTime=" + initRequestedTime
+        + ", initedTime=" + initedTime
+        + ", startRequestedTime=" + startRequestedTime
+        + ", startedTime=" + startTime
+        + ", finishTime=" + finishTime
+        + ", timeTaken=" + (finishTime - startTime)
+        + ", status=" + state.name()
+        + ", diagnostics=" + diagnostics
         + ", counters="
         + tezCounters.toString()
             .replaceAll("\\n", ", ").replaceAll("\\s+", " ");
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..21c7587
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.history.events;
+
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public class VertexInitializedEvent implements HistoryEvent {
+
+  private TezVertexID vertexID;
+  private String vertexName;
+  private long initRequestedTime;
+  private long initedTime;
+  private long numTasks;
+  private String processorName;
+
+  public VertexInitializedEvent() {
+  }
+
+  public VertexInitializedEvent(TezVertexID vertexId,
+      String vertexName, long initRequestedTime, long initedTime,
+      long numTasks, String processorName) {
+    this.vertexName = vertexName;
+    this.vertexID = vertexId;
+    this.initRequestedTime = initRequestedTime;
+    this.initedTime = initedTime;
+    this.numTasks = numTasks;
+    this.processorName = processorName;
+  }
+
+  @Override
+  public HistoryEventType getEventType() {
+    return HistoryEventType.VERTEX_INITIALIZED;
+  }
+
+  @Override
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, vertexID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name());
+
+    // Related entities
+    JSONArray relatedEntities = new JSONArray();
+    JSONObject vertexEntity = new JSONObject();
+    vertexEntity.put(ATSConstants.ENTITY, vertexID.getDAGId().toString());
+    vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name());
+    relatedEntities.put(vertexEntity);
+    jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities);
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject initEvent = new JSONObject();
+    initEvent.put(ATSConstants.TIMESTAMP, initedTime);
+    initEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.VERTEX_INITIALIZED.name());
+    events.put(initEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    // Other info
+    // TODO fix requested times to be events
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.VERTEX_NAME, vertexName);
+    otherInfo.put(ATSConstants.INIT_REQUESTED_TIME, initRequestedTime);
+    otherInfo.put(ATSConstants.INIT_TIME, initedTime);
+    otherInfo.put(ATSConstants.NUM_TASKS, numTasks);
+    otherInfo.put(ATSConstants.PROCESSOR_CLASS_NAME, processorName);
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public RecoveryProtos.VertexInitializedProto toProto() {
+    return RecoveryProtos.VertexInitializedProto.newBuilder()
+        .setVertexId(vertexID.toString())
+        .setVertexName(vertexName)
+        .setInitRequestedTime(initRequestedTime)
+        .setInitTime(initedTime)
+        .setNumTasks(numTasks)
+        .build();
+  }
+
+  public void fromProto(RecoveryProtos.VertexInitializedProto proto) {
+    this.vertexID = TezVertexID.fromString(proto.getVertexId());
+    this.vertexName = proto.getVertexName();
+    this.initRequestedTime = proto.getInitRequestedTime();
+    this.initedTime = proto.getInitTime();
+    this.numTasks = proto.getNumTasks();
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    RecoveryProtos.VertexInitializedProto proto =
+        RecoveryProtos.VertexInitializedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
+  }
+
+  @Override
+  public String toString() {
+    return "vertexName=" + vertexName
+        + ", vertexId=" + vertexID
+        + ", initRequestedTime=" + initRequestedTime
+        + ", initedTime=" + initedTime
+        + ", numTasks=" + numTasks
+        + ", processorName=" + processorName;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java
----------------------------------------------------------------------
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 6487b5c..6bb383d 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
@@ -19,25 +19,33 @@
 package org.apache.tez.dag.history.events;
 
 import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.avro.HistoryEventType;
-import org.apache.tez.dag.history.avro.VertexStarted;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.ats.EntityTypes;
+import org.apache.tez.dag.history.utils.ATSConstants;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexStartedProto;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 public class VertexStartedEvent implements HistoryEvent {
 
-  private VertexStarted datum = new VertexStarted();
+  private TezVertexID vertexID;
+  private long startRequestedTime;
+  private long startTime;
+
+  public VertexStartedEvent() {
+  }
 
   public VertexStartedEvent(TezVertexID vertexId,
-      String vertexName, long initRequestedTime, long initedTime, long startRequestedTime, long startTime,
-      long numTasks, String processorName) {
-    datum.vertexName = vertexName;
-    datum.vertexId = vertexId.toString();
-    datum.initRequestedTime = initRequestedTime;
-    datum.initedTime = initedTime;
-    datum.startRequestedTime = startRequestedTime;
-    datum.startedTime = startTime;
-    datum.numTasks = numTasks;
-    datum.processorName = processorName;
+      long startRequestedTime, long startTime) {
+    this.vertexID = vertexId;
+    this.startRequestedTime = startRequestedTime;
+    this.startTime = startTime;
   }
 
   @Override
@@ -46,25 +54,78 @@ public class VertexStartedEvent implements HistoryEvent {
   }
 
   @Override
-  public Object getBlob() {
-    // TODO Auto-generated method stub
-    return this.toString();
+  public JSONObject convertToATSJSON() throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, vertexID.toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name());
+
+    // Related entities
+    JSONArray relatedEntities = new JSONArray();
+    JSONObject vertexEntity = new JSONObject();
+    vertexEntity.put(ATSConstants.ENTITY, vertexID.getDAGId().toString());
+    vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name());
+    relatedEntities.put(vertexEntity);
+    jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities);
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject startEvent = new JSONObject();
+    startEvent.put(ATSConstants.TIMESTAMP, startTime);
+    startEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.VERTEX_STARTED.name());
+    events.put(startEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    // Other info
+    // TODO fix requested times to be events
+    JSONObject otherInfo = new JSONObject();
+    otherInfo.put(ATSConstants.START_REQUESTED_TIME, startRequestedTime);
+    otherInfo.put(ATSConstants.START_TIME, startTime);
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    return jsonObject;
   }
 
   @Override
-  public void setBlob(Object blob) {
-    this.datum = (VertexStarted) blob;
+  public boolean isRecoveryEvent() {
+    return false;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public VertexStartedProto toProto() {
+    return VertexStartedProto.newBuilder()
+        .setVertexId(vertexID.toString())
+        .setStartRequestedTime(startRequestedTime)
+        .setStartTime(startTime)
+        .build();
+  }
+
+  public void fromProto(VertexStartedProto proto) {
+    this.vertexID = TezVertexID.fromString(proto.getVertexId());
+    this.startRequestedTime = proto.getStartRequestedTime();
+    this.startTime = proto.getStartTime();
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    VertexStartedProto proto = VertexStartedProto.parseDelimitedFrom(inputStream);
+    fromProto(proto);
   }
 
   @Override
   public String toString() {
-    return "vertexName=" + datum.vertexName
-        + ", vertexId=" + datum.vertexId
-        + ", initRequestedTime=" + datum.initRequestedTime
-        + ", initedTime=" + datum.initedTime
-        + ", startRequestedTime=" + datum.startRequestedTime
-        + ", startedTime=" + datum.startedTime
-        + ", numTasks=" + datum.numTasks
-        + ", processorName=" + datum.processorName;
+    return "vertexId=" + vertexID
+        + ", startRequestedTime=" + startRequestedTime
+        + ", startedTime=" + startTime;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..807ad81
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
@@ -0,0 +1,267 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.history.recovery;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.SummaryEvent;
+import org.apache.tez.dag.history.events.DAGSubmittedEvent;
+import org.apache.tez.dag.records.TezDAGID;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class RecoveryService extends AbstractService {
+
+  private static final Log LOG = LogFactory.getLog(RecoveryService.class);
+  private final AppContext appContext;
+
+  private LinkedBlockingQueue<DAGHistoryEvent> eventQueue =
+      new LinkedBlockingQueue<DAGHistoryEvent>();
+  private Set<TezDAGID> completedDAGs = new HashSet<TezDAGID>();
+
+  private Thread eventHandlingThread;
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+  private AtomicBoolean started = new AtomicBoolean(false);
+  private int eventCounter = 0;
+  private int eventsProcessed = 0;
+  private final Object lock = new Object();
+  private FileSystem recoveryDirFS; // FS where staging dir exists
+  Path recoveryPath;
+  Map<TezDAGID, FSDataOutputStream> outputStreamMap = new
+      HashMap<TezDAGID, FSDataOutputStream>();
+  // FSDataOutputStream metaInfoStream;
+  private int bufferSize;
+  private FSDataOutputStream summaryStream;
+
+  public RecoveryService(AppContext appContext) {
+    super(RecoveryService.class.getName());
+    this.appContext = appContext;
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    LOG.info("Initializing RecoveryService");
+    recoveryPath = appContext.getCurrentRecoveryDir();
+    recoveryDirFS = FileSystem.get(recoveryPath.toUri(), conf);
+    bufferSize = conf.getInt(TezConfiguration.DAG_RECOVERY_FILE_IO_BUFFER_SIZE,
+        TezConfiguration.DAG_RECOVERY_FILE_IO_BUFFER_SIZE_DEFAULT);
+  }
+
+  @Override
+  public void serviceStart() {
+    LOG.info("Starting RecoveryService");
+    eventHandlingThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        DAGHistoryEvent event;
+        while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
+
+          // Log the size of the event-queue every so often.
+          if (eventCounter != 0 && eventCounter % 1000 == 0) {
+            LOG.info("Event queue stats"
+                + ", eventsProcessedSinceLastUpdate=" + eventsProcessed
+                + ", eventQueueSize=" + eventQueue.size());
+            eventCounter = 0;
+            eventsProcessed = 0;
+          } else {
+            ++eventCounter;
+          }
+
+          try {
+            event = eventQueue.take();
+          } catch (InterruptedException e) {
+            LOG.info("EventQueue take interrupted. Returning");
+            return;
+          }
+
+          synchronized (lock) {
+            try {
+              ++eventsProcessed;
+              handleEvent(event);
+            } catch (Exception e) {
+              // TODO handle failures - treat as fatal or ignore?
+              LOG.warn("Error handling recovery event", e);
+            }
+          }
+        }
+      }
+    }, "RecoveryEventHandlingThread");
+    eventHandlingThread.start();
+    started.set(true);
+  }
+
+  @Override
+  public void serviceStop() {
+    LOG.info("Stopping RecoveryService");
+    stopped.set(true);
+    if (eventHandlingThread != null) {
+      eventHandlingThread.interrupt();
+    }
+    if (summaryStream != null) {
+      try {
+        summaryStream.flush();
+        summaryStream.close();
+      } catch (IOException ioe) {
+        LOG.warn("Error when closing summary stream", ioe);
+      }
+    }
+    for (FSDataOutputStream outputStream : outputStreamMap.values()) {
+      try {
+        outputStream.flush();
+        outputStream.close();
+      } catch (IOException ioe) {
+        LOG.warn("Error when closing output stream", ioe);
+      }
+    }
+  }
+
+  public void handle(DAGHistoryEvent event) {
+    if (stopped.get()) {
+      LOG.warn("Igoring event as service stopped, eventType"
+          + event.getHistoryEvent().getEventType());
+      return;
+    }
+    if (!started.get()) {
+      eventQueue.add(event);
+      return;
+    }
+    HistoryEventType eventType = event.getHistoryEvent().getEventType();
+    if (eventType.equals(HistoryEventType.DAG_SUBMITTED)
+      || eventType.equals(HistoryEventType.DAG_FINISHED)) {
+      // handle submissions and completion immediately
+      synchronized (lock) {
+        try {
+          handleEvent(event);
+          summaryStream.flush();
+          if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) {
+            outputStreamMap.get(event.getDagID()).flush();
+          } else if (eventType.equals(HistoryEventType.DAG_FINISHED)) {
+            completedDAGs.add(event.getDagID());
+            if (outputStreamMap.containsKey(event.getDagID())) {
+              try {
+                outputStreamMap.get(event.getDagID()).flush();
+                outputStreamMap.get(event.getDagID()).close();
+                outputStreamMap.remove(event.getDagID());
+              } catch (IOException ioe) {
+                LOG.warn("Error when trying to flush/close recovery file for"
+                    + " dag, dagId=" + event.getDagID());
+              }
+            } else {
+              // TODO this is an error
+            }
+          }
+        } catch (Exception e) {
+            // TODO handle failures - treat as fatal or ignore?
+            LOG.warn("Error handling recovery event", e);
+        }
+      }
+      LOG.info("DAG completed"
+          + ", dagId=" + event.getDagID()
+          + ", queueSize=" + eventQueue.size());
+    } else {
+      // All other events just get queued
+      eventQueue.add(event);
+    }
+  }
+
+
+  private void handleEvent(DAGHistoryEvent event) {
+    HistoryEventType eventType = event.getHistoryEvent().getEventType();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Handling recovery event of type "
+          + event.getHistoryEvent().getEventType());
+    }
+    if (event.getDagID() == null) {
+      // AM event
+      // anything to be done?
+      // TODO
+      return;
+    }
+
+    TezDAGID dagID = event.getDagID();
+    if (completedDAGs.contains(dagID)) {
+      // Skip events for completed DAGs
+      // no need to recover completed DAGs
+      return;
+    }
+
+    try {
+
+      if (eventType.equals(HistoryEventType.DAG_SUBMITTED)
+          || eventType.equals(HistoryEventType.DAG_FINISHED)) {
+        if (summaryStream == null) {
+          Path summaryPath = new Path(recoveryPath,
+              appContext.getApplicationID()
+              + TezConfiguration.DAG_RECOVERY_SUMMARY_FILE_SUFFIX);
+          summaryStream = recoveryDirFS.create(summaryPath, false,
+              bufferSize);
+        }
+        if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) {
+          DAGSubmittedEvent dagSubmittedEvent =
+              (DAGSubmittedEvent) event.getHistoryEvent();
+          String dagName = dagSubmittedEvent.getDAGName();
+          if (dagName != null
+              && dagName.startsWith(
+              TezConfiguration.TEZ_PREWARM_DAG_NAME_PREFIX)) {
+            // Skip recording pre-warm DAG events
+            return;
+          }
+          Path dagFilePath = new Path(recoveryPath,
+              dagID.toString() + TezConfiguration.DAG_RECOVERY_RECOVER_FILE_SUFFIX);
+          FSDataOutputStream outputStream =
+              recoveryDirFS.create(dagFilePath, false, bufferSize);
+          outputStreamMap.put(dagID, outputStream);
+        }
+
+        if (outputStreamMap.containsKey(dagID)) {
+          SummaryEvent summaryEvent = (SummaryEvent) event.getHistoryEvent();
+          summaryEvent.toSummaryProtoStream(summaryStream);
+        }
+      }
+
+      FSDataOutputStream outputStream = outputStreamMap.get(dagID);
+      if (outputStream == null) {
+        return;
+      }
+
+      outputStream.write(event.getHistoryEvent().getEventType().ordinal());
+      event.getHistoryEvent().toProtoStream(outputStream);
+    } catch (IOException ioe) {
+      // TODO handle failures - treat as fatal or ignore?
+      LOG.warn("Failed to write to stream", ioe);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/14127af1/tez-dag/src/main/java/org/apache/tez/dag/history/utils/ATSConstants.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/ATSConstants.java b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/ATSConstants.java
new file mode 100644
index 0000000..8f680ce
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/ATSConstants.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.history.utils;
+
+public class ATSConstants {
+
+  // TODO remove once YARN exposes proper constants
+
+  /* Top level keys */
+  public static final String ENTITY = "entity";
+  public static final String ENTITY_TYPE = "entitytype";
+  public static final String EVENTS = "events";
+  public static final String EVENT_TYPE = "eventtype";
+  public static final String TIMESTAMP = "ts";
+  public static final String EVENT_INFO = "eventinfo";
+  public static final String RELATED_ENTITIES = "relatedEntities";
+  public static final String PRIMARY_FILTERS = "primaryfilters";
+  public static final String SECONDARY_FILTERS = "secondaryfilters";
+  public static final String OTHER_INFO = "otherinfo";
+
+  /* Section for related entities */
+  public static final String APPLICATION_ID = "applicationId";
+  public static final String APPLICATION_ATTEMPT_ID = "applicationAttemptId";
+  public static final String CONTAINER_ID = "containerId";
+  public static final String NODE_ID = "nodeId";
+
+  /* Keys used in other info */
+  public static final String APP_SUBMIT_TIME = "appSubmitTime";
+
+  /* Tez-specific info */
+  public static final String DAG_PLAN = "dagPlan";
+  public static final String DAG_NAME = "dagName";
+  public static final String VERTEX_NAME = "vertexName";
+  public static final String SCHEDULED_TIME = "scheduledTime";
+  public static final String INIT_REQUESTED_TIME = "initRequestedTime";
+  public static final String INIT_TIME = "initTime";
+  public static final String START_REQUESTED_TIME = "startRequestedTime";
+  public static final String START_TIME = "startTime";
+  public static final String FINISH_TIME = "endTime";
+  public static final String TIME_TAKEN = "timeTaken";
+  public static final String STATUS = "status";
+  public static final String DIAGNOSTICS = "diagnostics";
+  public static final String COUNTERS = "counters";
+  public static final String NUM_TASKS = "numTasks";
+  public static final String PROCESSOR_CLASS_NAME = "processorClassName";
+  public static final String IN_PROGRESS_LOGS_URL = "inProgressLogsURL";
+  public static final String COMPLETED_LOGS_URL = "completedLogsURL";
+
+  /* Counters-related keys */
+  public static final String COUNTER_GROUPS = "counterGroups";
+  public static final String COUNTER_GROUP_NAME = "counterGroupName";
+  public static final String COUNTER_GROUP_DISPLAY_NAME = "counterGroupDisplayName";
+  public static final String COUNTER_NAME = "counterName";
+  public static final String COUNTER_DISPLAY_NAME = "counterDisplayName";
+  public static final String COUNTER_VALUE = "counterValue";
+
+}


Mime
View raw message