Browse Source

MAPREDUCE-6388. Remove deprecation warnings from JobHistoryServer classes. Contributed by Ray Chiang.

(cherry picked from commit 6afe20a7a4bbfa30fce3e3c9873ad43201987998)
Tsuyoshi Ozawa 10 years ago
parent
commit
c6cdecc6b3
24 changed files with 393 additions and 374 deletions
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 74 74
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
  3. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java
  4. 12 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
  5. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AvroArrayUtils.java
  6. 12 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java
  7. 12 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventWriter.java
  8. 20 19
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
  9. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java
  10. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
  11. 12 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
  12. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
  13. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
  14. 31 31
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
  15. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
  16. 33 33
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
  17. 35 35
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
  18. 17 17
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
  19. 26 23
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
  20. 33 33
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
  21. 16 16
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
  22. 13 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
  23. 12 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
  24. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -201,6 +201,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6377. JHS sorting on state column not working in webUi.
     (zhihai xu via devaraj)
 
+    MAPREDUCE-6388. Remove deprecation warnings from JobHistoryServer classes
+    (Ray Chiang via ozawa).
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 74 - 74
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java

@@ -120,65 +120,65 @@ public class TestEvents {
         new ByteArrayInputStream(getEvents())));
     HistoryEvent e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.JOB_PRIORITY_CHANGED));
-    assertEquals("ID", ((JobPriorityChange) e.getDatum()).jobid.toString());
+    assertEquals("ID", ((JobPriorityChange) e.getDatum()).getJobid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.JOB_STATUS_CHANGED));
-    assertEquals("ID", ((JobStatusChanged) e.getDatum()).jobid.toString());
+    assertEquals("ID", ((JobStatusChanged) e.getDatum()).getJobid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.TASK_UPDATED));
-    assertEquals("ID", ((TaskUpdated) e.getDatum()).taskid.toString());
+    assertEquals("ID", ((TaskUpdated) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
     assertEquals(taskId,
-        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
+        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.JOB_KILLED));
     assertEquals("ID",
-        ((JobUnsuccessfulCompletion) e.getDatum()).jobid.toString());
+        ((JobUnsuccessfulCompletion) e.getDatum()).getJobid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_STARTED));
     assertEquals(taskId,
-        ((TaskAttemptStarted) e.getDatum()).taskid.toString());
+        ((TaskAttemptStarted) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_FINISHED));
     assertEquals(taskId,
-        ((TaskAttemptFinished) e.getDatum()).taskid.toString());
+        ((TaskAttemptFinished) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
     assertEquals(taskId,
-        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
+        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
     assertEquals(taskId,
-        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
+        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_STARTED));
     assertEquals(taskId,
-        ((TaskAttemptStarted) e.getDatum()).taskid.toString());
+        ((TaskAttemptStarted) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_FINISHED));
     assertEquals(taskId,
-        ((TaskAttemptFinished) e.getDatum()).taskid.toString());
+        ((TaskAttemptFinished) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
     assertEquals(taskId,
-        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
+        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).getTaskid().toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
     assertEquals(taskId,
-        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
+        ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).getTaskid().toString());
 
     reader.close();
   }
@@ -227,43 +227,43 @@ public class TestEvents {
 
   private TaskAttemptUnsuccessfulCompletion getTaskAttemptUnsuccessfulCompletion() {
     TaskAttemptUnsuccessfulCompletion datum = new TaskAttemptUnsuccessfulCompletion();
-    datum.attemptId = "attempt_1_2_r3_4_5";
-    datum.clockSplits = Arrays.asList(1, 2, 3);
-    datum.cpuUsages = Arrays.asList(100, 200, 300);
-    datum.error = "Error";
-    datum.finishTime = 2;
-    datum.hostname = "hostname";
-    datum.rackname = "rackname";
-    datum.physMemKbytes = Arrays.asList(1000, 2000, 3000);
-    datum.taskid = taskId;
-    datum.port = 1000;
-    datum.taskType = "REDUCE";
-    datum.status = "STATUS";
-    datum.counters = getCounters();
-    datum.vMemKbytes = Arrays.asList(1000, 2000, 3000);
+    datum.setAttemptId("attempt_1_2_r3_4_5");
+    datum.setClockSplits(Arrays.asList(1, 2, 3));
+    datum.setCpuUsages(Arrays.asList(100, 200, 300));
+    datum.setError("Error");
+    datum.setFinishTime(2L);
+    datum.setHostname("hostname");
+    datum.setRackname("rackname");
+    datum.setPhysMemKbytes(Arrays.asList(1000, 2000, 3000));
+    datum.setTaskid(taskId);
+    datum.setPort(1000);
+    datum.setTaskType("REDUCE");
+    datum.setStatus("STATUS");
+    datum.setCounters(getCounters());
+    datum.setVMemKbytes(Arrays.asList(1000, 2000, 3000));
     return datum;
   }
 
   private JhCounters getCounters() {
     JhCounters counters = new JhCounters();
-    counters.groups = new ArrayList<JhCounterGroup>(0);
-    counters.name = "name";
+    counters.setGroups(new ArrayList<JhCounterGroup>(0));
+    counters.setName("name");
     return counters;
   }
 
   private FakeEvent getCleanupAttemptFinishedEvent() {
     FakeEvent result = new FakeEvent(EventType.CLEANUP_ATTEMPT_FINISHED);
     TaskAttemptFinished datum = new TaskAttemptFinished();
-    datum.attemptId = "attempt_1_2_r3_4_5";
-
-    datum.counters = getCounters();
-    datum.finishTime = 2;
-    datum.hostname = "hostname";
-    datum.rackname = "rackName";
-    datum.state = "state";
-    datum.taskid = taskId;
-    datum.taskStatus = "taskStatus";
-    datum.taskType = "REDUCE";
+    datum.setAttemptId("attempt_1_2_r3_4_5");
+
+    datum.setCounters(getCounters());
+    datum.setFinishTime(2L);
+    datum.setHostname("hostname");
+    datum.setRackname("rackName");
+    datum.setState("state");
+    datum.setTaskid(taskId);
+    datum.setTaskStatus("taskStatus");
+    datum.setTaskType("REDUCE");
     result.setDatum(datum);
     return result;
   }
@@ -272,16 +272,16 @@ public class TestEvents {
     FakeEvent result = new FakeEvent(EventType.CLEANUP_ATTEMPT_STARTED);
     TaskAttemptStarted datum = new TaskAttemptStarted();
 
-    datum.attemptId = "attempt_1_2_r3_4_5";
-    datum.avataar = "avatar";
-    datum.containerId = "containerId";
-    datum.httpPort = 10000;
-    datum.locality = "locality";
-    datum.shufflePort = 10001;
-    datum.startTime = 1;
-    datum.taskid = taskId;
-    datum.taskType = "taskType";
-    datum.trackerName = "trackerName";
+    datum.setAttemptId("attempt_1_2_r3_4_5");
+    datum.setAvataar("avatar");
+    datum.setContainerId("containerId");
+    datum.setHttpPort(10000);
+    datum.setLocality("locality");
+    datum.setShufflePort(10001);
+    datum.setStartTime(1L);
+    datum.setTaskid(taskId);
+    datum.setTaskType("taskType");
+    datum.setTrackerName("trackerName");
     result.setDatum(datum);
     return result;
   }
@@ -303,15 +303,15 @@ public class TestEvents {
     FakeEvent result = new FakeEvent(EventType.SETUP_ATTEMPT_FINISHED);
     TaskAttemptFinished datum = new TaskAttemptFinished();
 
-    datum.attemptId = "attempt_1_2_r3_4_5";
-    datum.counters = getCounters();
-    datum.finishTime = 2;
-    datum.hostname = "hostname";
-    datum.rackname = "rackname";
-    datum.state = "state";
-    datum.taskid = taskId;
-    datum.taskStatus = "taskStatus";
-    datum.taskType = "REDUCE";
+    datum.setAttemptId("attempt_1_2_r3_4_5");
+    datum.setCounters(getCounters());
+    datum.setFinishTime(2L);
+    datum.setHostname("hostname");
+    datum.setRackname("rackname");
+    datum.setState("state");
+    datum.setTaskid(taskId);
+    datum.setTaskStatus("taskStatus");
+    datum.setTaskType("REDUCE");
     result.setDatum(datum);
     return result;
   }
@@ -319,16 +319,16 @@ public class TestEvents {
   private FakeEvent getSetupAttemptStartedEvent() {
     FakeEvent result = new FakeEvent(EventType.SETUP_ATTEMPT_STARTED);
     TaskAttemptStarted datum = new TaskAttemptStarted();
-    datum.attemptId = "ID";
-    datum.avataar = "avataar";
-    datum.containerId = "containerId";
-    datum.httpPort = 10000;
-    datum.locality = "locality";
-    datum.shufflePort = 10001;
-    datum.startTime = 1;
-    datum.taskid = taskId;
-    datum.taskType = "taskType";
-    datum.trackerName = "trackerName";
+    datum.setAttemptId("ID");
+    datum.setAvataar("avataar");
+    datum.setContainerId("containerId");
+    datum.setHttpPort(10000);
+    datum.setLocality("locality");
+    datum.setShufflePort(10001);
+    datum.setStartTime(1L);
+    datum.setTaskid(taskId);
+    datum.setTaskType("taskType");
+    datum.setTrackerName("trackerName");
     result.setDatum(datum);
     return result;
   }
@@ -356,8 +356,8 @@ public class TestEvents {
   private FakeEvent getJobPriorityChangedEvent() {
     FakeEvent result = new FakeEvent(EventType.JOB_PRIORITY_CHANGED);
     JobPriorityChange datum = new JobPriorityChange();
-    datum.jobid = "ID";
-    datum.priority = "priority";
+    datum.setJobid("ID");
+    datum.setPriority("priority");
     result.setDatum(datum);
     return result;
   }
@@ -365,8 +365,8 @@ public class TestEvents {
   private FakeEvent getJobStatusChangedEvent() {
     FakeEvent result = new FakeEvent(EventType.JOB_STATUS_CHANGED);
     JobStatusChanged datum = new JobStatusChanged();
-    datum.jobid = "ID";
-    datum.jobStatus = "newStatus";
+    datum.setJobid("ID");
+    datum.setJobStatus("newStatus");
     result.setDatum(datum);
     return result;
   }
@@ -374,8 +374,8 @@ public class TestEvents {
   private FakeEvent getTaskUpdatedEvent() {
     FakeEvent result = new FakeEvent(EventType.TASK_UPDATED);
     TaskUpdated datum = new TaskUpdated();
-    datum.finishTime = 2;
-    datum.taskid = "ID";
+    datum.setFinishTime(2L);
+    datum.setTaskid("ID");
     result.setDatum(datum);
     return result;
   }

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java

@@ -38,11 +38,11 @@ public class TestJobSummary {
     JobId mockJobId = mock(JobId.class);
     when(mockJobId.toString()).thenReturn("testJobId");
     summary.setJobId(mockJobId);
-    summary.setJobSubmitTime(2);
-    summary.setJobLaunchTime(3);
-    summary.setFirstMapTaskLaunchTime(4);
-    summary.setFirstReduceTaskLaunchTime(5);
-    summary.setJobFinishTime(6);
+    summary.setJobSubmitTime(2L);
+    summary.setJobLaunchTime(3L);
+    summary.setFirstMapTaskLaunchTime(4L);
+    summary.setFirstReduceTaskLaunchTime(5L);
+    summary.setJobFinishTime(6L);
     summary.setNumFinishedMaps(1);
     summary.setNumFailedMaps(0);
     summary.setNumFinishedReduces(1);

+ 12 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java

@@ -82,12 +82,12 @@ public class AMStartedEvent implements HistoryEvent {
       ContainerId containerId, String nodeManagerHost, int nodeManagerPort,
       int nodeManagerHttpPort, String forcedJobStateOnShutDown,
       long submitTime) {
-    datum.applicationAttemptId = new Utf8(appAttemptId.toString());
-    datum.startTime = startTime;
-    datum.containerId = new Utf8(containerId.toString());
-    datum.nodeManagerHost = new Utf8(nodeManagerHost);
-    datum.nodeManagerPort = nodeManagerPort;
-    datum.nodeManagerHttpPort = nodeManagerHttpPort;
+    datum.setApplicationAttemptId(new Utf8(appAttemptId.toString()));
+    datum.setStartTime(startTime);
+    datum.setContainerId(new Utf8(containerId.toString()));
+    datum.setNodeManagerHost(new Utf8(nodeManagerHost));
+    datum.setNodeManagerPort(nodeManagerPort);
+    datum.setNodeManagerHttpPort(nodeManagerHttpPort);
     this.forcedJobStateOnShutDown = forcedJobStateOnShutDown;
     this.submitTime = submitTime;
   }
@@ -107,7 +107,7 @@ public class AMStartedEvent implements HistoryEvent {
    * @return the ApplicationAttemptId
    */
   public ApplicationAttemptId getAppAttemptId() {
-    return ConverterUtils.toApplicationAttemptId(datum.applicationAttemptId
+    return ConverterUtils.toApplicationAttemptId(datum.getApplicationAttemptId()
         .toString());
   }
 
@@ -115,35 +115,35 @@ public class AMStartedEvent implements HistoryEvent {
    * @return the start time for the MRAppMaster
    */
   public long getStartTime() {
-    return datum.startTime;
+    return datum.getStartTime();
   }
 
   /**
    * @return the ContainerId for the MRAppMaster.
    */
   public ContainerId getContainerId() {
-    return ConverterUtils.toContainerId(datum.containerId.toString());
+    return ConverterUtils.toContainerId(datum.getContainerId().toString());
   }
 
   /**
    * @return the node manager host.
    */
   public String getNodeManagerHost() {
-    return datum.nodeManagerHost.toString();
+    return datum.getNodeManagerHost().toString();
   }
 
   /**
    * @return the node manager port.
    */
   public int getNodeManagerPort() {
-    return datum.nodeManagerPort;
+    return datum.getNodeManagerPort();
   }
   
   /**
    * @return the http port for the tracker.
    */
   public int getNodeManagerHttpPort() {
-    return datum.nodeManagerHttpPort;
+    return datum.getNodeManagerHttpPort();
   }
 
   /**

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AvroArrayUtils.java

@@ -45,7 +45,7 @@ public class AvroArrayUtils {
   }
 
   public static int[] fromAvro(List<Integer> avro) {
-    int[] result = new int[(int)avro.size()];
+    int[] result = new int[avro.size()];
 
     int i = 0;
       

+ 12 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java

@@ -72,7 +72,8 @@ public class EventReader implements Closeable {
     }
 
     Schema myschema = new SpecificData(Event.class.getClassLoader()).getSchema(Event.class);
-    this.schema = Schema.parse(in.readLine());
+    Schema.Parser parser = new Schema.Parser();
+    this.schema = parser.parse(in.readLine());
     this.reader = new SpecificDatumReader(schema, myschema);
     this.decoder = DecoderFactory.get().jsonDecoder(schema, in);
   }
@@ -91,7 +92,7 @@ public class EventReader implements Closeable {
       return null;
     }
     HistoryEvent result;
-    switch (wrapper.type) {
+    switch (wrapper.getType()) {
     case JOB_SUBMITTED:
       result = new JobSubmittedEvent(); break;
     case JOB_INITED:
@@ -155,9 +156,9 @@ public class EventReader implements Closeable {
     case AM_STARTED:
       result = new AMStartedEvent(); break;
     default:
-      throw new RuntimeException("unexpected event type: " + wrapper.type);
+      throw new RuntimeException("unexpected event type: " + wrapper.getType());
     }
-    result.setDatum(wrapper.event);
+    result.setDatum(wrapper.getEvent());
     return result;
   }
 
@@ -176,13 +177,14 @@ public class EventReader implements Closeable {
   static Counters fromAvro(JhCounters counters) {
     Counters result = new Counters();
     if(counters != null) {
-      for (JhCounterGroup g : counters.groups) {
+      for (JhCounterGroup g : counters.getGroups()) {
         CounterGroup group =
-            result.addGroup(StringInterner.weakIntern(g.name.toString()), 
-                StringInterner.weakIntern(g.displayName.toString()));
-        for (JhCounter c : g.counts) {
-          group.addCounter(StringInterner.weakIntern(c.name.toString()), 
-              StringInterner.weakIntern(c.displayName.toString()), c.value);
+            result.addGroup(StringInterner.weakIntern(g.getName().toString()),
+                StringInterner.weakIntern(g.getDisplayName().toString()));
+        for (JhCounter c : g.getCounts()) {
+          group.addCounter(StringInterner.weakIntern(c.getName().toString()),
+              StringInterner.weakIntern(c.getDisplayName().toString()),
+                  c.getValue());
         }
       }
     }

+ 12 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventWriter.java

@@ -61,8 +61,8 @@ class EventWriter {
   
   synchronized void write(HistoryEvent event) throws IOException { 
     Event wrapper = new Event();
-    wrapper.type = event.getEventType();
-    wrapper.event = event.getDatum();
+    wrapper.setType(event.getEventType());
+    wrapper.setEvent(event.getDatum());
     writer.write(wrapper, encoder);
     encoder.flush();
     out.writeBytes("\n");
@@ -95,22 +95,22 @@ class EventWriter {
   }
   static JhCounters toAvro(Counters counters, String name) {
     JhCounters result = new JhCounters();
-    result.name = new Utf8(name);
-    result.groups = new ArrayList<JhCounterGroup>(0);
+    result.setName(new Utf8(name));
+    result.setGroups(new ArrayList<JhCounterGroup>(0));
     if (counters == null) return result;
     for (CounterGroup group : counters) {
       JhCounterGroup g = new JhCounterGroup();
-      g.name = new Utf8(group.getName());
-      g.displayName = new Utf8(group.getDisplayName());
-      g.counts = new ArrayList<JhCounter>(group.size());
+      g.setName(new Utf8(group.getName()));
+      g.setDisplayName(new Utf8(group.getDisplayName()));
+      g.setCounts(new ArrayList<JhCounter>(group.size()));
       for (Counter counter : group) {
         JhCounter c = new JhCounter();
-        c.name = new Utf8(counter.getName());
-        c.displayName = new Utf8(counter.getDisplayName());
-        c.value = counter.getValue();
-        g.counts.add(c);
+        c.setName(new Utf8(counter.getName()));
+        c.setDisplayName(new Utf8(counter.getDisplayName()));
+        c.setValue(counter.getValue());
+        g.getCounts().add(c);
       }
-      result.groups.add(g);
+      result.getGroups().add(g);
     }
     return result;
   }

+ 20 - 19
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java

@@ -77,31 +77,32 @@ public class JobFinishedEvent  implements HistoryEvent {
   public Object getDatum() {
     if (datum == null) {
       datum = new JobFinished();
-      datum.jobid = new Utf8(jobId.toString());
-      datum.finishTime = finishTime;
-      datum.finishedMaps = finishedMaps;
-      datum.finishedReduces = finishedReduces;
-      datum.failedMaps = failedMaps;
-      datum.failedReduces = failedReduces;
-      datum.mapCounters = EventWriter.toAvro(mapCounters, "MAP_COUNTERS");
-      datum.reduceCounters = EventWriter.toAvro(reduceCounters,
-        "REDUCE_COUNTERS");
-      datum.totalCounters = EventWriter.toAvro(totalCounters, "TOTAL_COUNTERS");
+      datum.setJobid(new Utf8(jobId.toString()));
+      datum.setFinishTime(finishTime);
+      datum.setFinishedMaps(finishedMaps);
+      datum.setFinishedReduces(finishedReduces);
+      datum.setFailedMaps(failedMaps);
+      datum.setFailedReduces(failedReduces);
+      datum.setMapCounters(EventWriter.toAvro(mapCounters, "MAP_COUNTERS"));
+      datum.setReduceCounters(EventWriter.toAvro(reduceCounters,
+          "REDUCE_COUNTERS"));
+      datum.setTotalCounters(EventWriter.toAvro(totalCounters,
+          "TOTAL_COUNTERS"));
     }
     return datum;
   }
 
   public void setDatum(Object oDatum) {
     this.datum = (JobFinished) oDatum;
-    this.jobId = JobID.forName(datum.jobid.toString());
-    this.finishTime = datum.finishTime;
-    this.finishedMaps = datum.finishedMaps;
-    this.finishedReduces = datum.finishedReduces;
-    this.failedMaps = datum.failedMaps;
-    this.failedReduces = datum.failedReduces;
-    this.mapCounters = EventReader.fromAvro(datum.mapCounters);
-    this.reduceCounters = EventReader.fromAvro(datum.reduceCounters);
-    this.totalCounters = EventReader.fromAvro(datum.totalCounters);
+    this.jobId = JobID.forName(datum.getJobid().toString());
+    this.finishTime = datum.getFinishTime();
+    this.finishedMaps = datum.getFinishedMaps();
+    this.finishedReduces = datum.getFinishedReduces();
+    this.failedMaps = datum.getFailedMaps();
+    this.failedReduces = datum.getFailedReduces();
+    this.mapCounters = EventReader.fromAvro(datum.getMapCounters());
+    this.reduceCounters = EventReader.fromAvro(datum.getReduceCounters());
+    this.totalCounters = EventReader.fromAvro(datum.getTotalCounters());
   }
 
   public EventType getEventType() {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java

@@ -565,7 +565,7 @@ public class JobHistoryParser implements HistoryEventHandler {
     /** @return the AMInfo for the job's AppMaster */
     public List<AMInfo> getAMInfos() { return amInfos; }
     /** @return the AMInfo for the newest AppMaster */
-    public AMInfo getLatestAMInfo() { return latestAmInfo; };
+    public AMInfo getLatestAMInfo() { return latestAmInfo; }
   }
   
   /**

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java

@@ -42,9 +42,9 @@ public class JobInfoChangeEvent implements HistoryEvent {
    * @param launchTime Launch time of the job
    */
   public JobInfoChangeEvent(JobID id, long submitTime, long launchTime) {
-    datum.jobid = new Utf8(id.toString());
-    datum.submitTime = submitTime;
-    datum.launchTime = launchTime;
+    datum.setJobid(new Utf8(id.toString()));
+    datum.setSubmitTime(submitTime);
+    datum.setLaunchTime(launchTime);
   }
 
   JobInfoChangeEvent() { }
@@ -55,11 +55,11 @@ public class JobInfoChangeEvent implements HistoryEvent {
   }
 
   /** Get the Job ID */
-  public JobID getJobId() { return JobID.forName(datum.jobid.toString()); }
+  public JobID getJobId() { return JobID.forName(datum.getJobid().toString()); }
   /** Get the Job submit time */
-  public long getSubmitTime() { return datum.submitTime; }
+  public long getSubmitTime() { return datum.getSubmitTime(); }
   /** Get the Job launch time */
-  public long getLaunchTime() { return datum.launchTime; }
+  public long getLaunchTime() { return datum.getLaunchTime(); }
 
   public EventType getEventType() {
     return EventType.JOB_INFO_CHANGED;

+ 12 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java

@@ -44,12 +44,12 @@ public class JobInitedEvent implements HistoryEvent {
    */
   public JobInitedEvent(JobID id, long launchTime, int totalMaps,
                         int totalReduces, String jobStatus, boolean uberized) {
-    datum.jobid = new Utf8(id.toString());
-    datum.launchTime = launchTime;
-    datum.totalMaps = totalMaps;
-    datum.totalReduces = totalReduces;
-    datum.jobStatus = new Utf8(jobStatus);
-    datum.uberized = uberized;
+    datum.setJobid(new Utf8(id.toString()));
+    datum.setLaunchTime(launchTime);
+    datum.setTotalMaps(totalMaps);
+    datum.setTotalReduces(totalReduces);
+    datum.setJobStatus(new Utf8(jobStatus));
+    datum.setUberized(uberized);
   }
 
   JobInitedEvent() { }
@@ -58,19 +58,19 @@ public class JobInitedEvent implements HistoryEvent {
   public void setDatum(Object datum) { this.datum = (JobInited)datum; }
 
   /** Get the job ID */
-  public JobID getJobId() { return JobID.forName(datum.jobid.toString()); }
+  public JobID getJobId() { return JobID.forName(datum.getJobid().toString()); }
   /** Get the launch time */
-  public long getLaunchTime() { return datum.launchTime; }
+  public long getLaunchTime() { return datum.getLaunchTime(); }
   /** Get the total number of maps */
-  public int getTotalMaps() { return datum.totalMaps; }
+  public int getTotalMaps() { return datum.getTotalMaps(); }
   /** Get the total number of reduces */
-  public int getTotalReduces() { return datum.totalReduces; }
+  public int getTotalReduces() { return datum.getTotalReduces(); }
   /** Get the status */
-  public String getStatus() { return datum.jobStatus.toString(); }
+  public String getStatus() { return datum.getJobStatus().toString(); }
   /** Get the event type */
   public EventType getEventType() {
     return EventType.JOB_INITED;
   }
   /** Get whether the job's map and reduce stages were combined */
-  public boolean getUberized() { return datum.uberized; }
+  public boolean getUberized() { return datum.getUberized(); }
 }

+ 6 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java

@@ -41,8 +41,8 @@ public class JobPriorityChangeEvent implements HistoryEvent {
    * @param priority The new priority of the job
    */
   public JobPriorityChangeEvent(JobID id, JobPriority priority) {
-    datum.jobid = new Utf8(id.toString());
-    datum.priority = new Utf8(priority.name());
+    datum.setJobid(new Utf8(id.toString()));
+    datum.setPriority(new Utf8(priority.name()));
   }
 
   JobPriorityChangeEvent() { }
@@ -53,10 +53,12 @@ public class JobPriorityChangeEvent implements HistoryEvent {
   }
 
   /** Get the Job ID */
-  public JobID getJobId() { return JobID.forName(datum.jobid.toString()); }
+  public JobID getJobId() {
+    return JobID.forName(datum.getJobid().toString());
+  }
   /** Get the job priority */
   public JobPriority getPriority() {
-    return JobPriority.valueOf(datum.priority.toString());
+    return JobPriority.valueOf(datum.getPriority().toString());
   }
   /** Get the event type */
   public EventType getEventType() {

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java

@@ -41,8 +41,8 @@ public class JobStatusChangedEvent implements HistoryEvent {
    * @param jobStatus The new job status
    */
   public JobStatusChangedEvent(JobID id, String jobStatus) {
-    datum.jobid = new Utf8(id.toString());
-    datum.jobStatus = new Utf8(jobStatus);
+    datum.setJobid(new Utf8(id.toString()));
+    datum.setJobStatus(new Utf8(jobStatus));
   }
 
   JobStatusChangedEvent() {}
@@ -53,9 +53,9 @@ public class JobStatusChangedEvent implements HistoryEvent {
   }
 
   /** Get the Job Id */
-  public JobID getJobId() { return JobID.forName(datum.jobid.toString()); }
+  public JobID getJobId() { return JobID.forName(datum.getJobid().toString()); }
   /** Get the event status */
-  public String getStatus() { return datum.jobStatus.toString(); }
+  public String getStatus() { return datum.getJobStatus().toString(); }
   /** Get the event type */
   public EventType getEventType() {
     return EventType.JOB_STATUS_CHANGED;

+ 31 - 31
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java

@@ -100,34 +100,34 @@ public class JobSubmittedEvent implements HistoryEvent {
         Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
         String workflowId, String workflowName, String workflowNodeName,
         String workflowAdjacencies, String workflowTags) {
-    datum.jobid = new Utf8(id.toString());
-    datum.jobName = new Utf8(jobName);
-    datum.userName = new Utf8(userName);
-    datum.submitTime = submitTime;
-    datum.jobConfPath = new Utf8(jobConfPath);
+    datum.setJobid(new Utf8(id.toString()));
+    datum.setJobName(new Utf8(jobName));
+    datum.setUserName(new Utf8(userName));
+    datum.setSubmitTime(submitTime);
+    datum.setJobConfPath(new Utf8(jobConfPath));
     Map<CharSequence, CharSequence> jobAcls = new HashMap<CharSequence, CharSequence>();
     for (Entry<JobACL, AccessControlList> entry : jobACLs.entrySet()) {
       jobAcls.put(new Utf8(entry.getKey().getAclName()), new Utf8(
           entry.getValue().getAclString()));
     }
-    datum.acls = jobAcls;
+    datum.setAcls(jobAcls);
     if (jobQueueName != null) {
-      datum.jobQueueName = new Utf8(jobQueueName);
+      datum.setJobQueueName(new Utf8(jobQueueName));
     }
     if (workflowId != null) {
-      datum.workflowId = new Utf8(workflowId);
+      datum.setWorkflowId(new Utf8(workflowId));
     }
     if (workflowName != null) {
-      datum.workflowName = new Utf8(workflowName);
+      datum.setWorkflowName(new Utf8(workflowName));
     }
     if (workflowNodeName != null) {
-      datum.workflowNodeName = new Utf8(workflowNodeName);
+      datum.setWorkflowNodeName(new Utf8(workflowNodeName));
     }
     if (workflowAdjacencies != null) {
-      datum.workflowAdjacencies = new Utf8(workflowAdjacencies);
+      datum.setWorkflowAdjacencies(new Utf8(workflowAdjacencies));
     }
     if (workflowTags != null) {
-      datum.workflowTags = new Utf8(workflowTags);
+      datum.setWorkflowTags(new Utf8(workflowTags));
     }
   }
 
@@ -139,30 +139,30 @@ public class JobSubmittedEvent implements HistoryEvent {
   }
 
   /** Get the Job Id */
-  public JobID getJobId() { return JobID.forName(datum.jobid.toString()); }
+  public JobID getJobId() { return JobID.forName(datum.getJobid().toString()); }
   /** Get the Job name */
-  public String getJobName() { return datum.jobName.toString(); }
+  public String getJobName() { return datum.getJobName().toString(); }
   /** Get the Job queue name */
   public String getJobQueueName() {
-    if (datum.jobQueueName != null) {
-      return datum.jobQueueName.toString();
+    if (datum.getJobQueueName() != null) {
+      return datum.getJobQueueName().toString();
     }
     return null;
   }
   /** Get the user name */
-  public String getUserName() { return datum.userName.toString(); }
+  public String getUserName() { return datum.getUserName().toString(); }
   /** Get the submit time */
-  public long getSubmitTime() { return datum.submitTime; }
+  public long getSubmitTime() { return datum.getSubmitTime(); }
   /** Get the Path for the Job Configuration file */
-  public String getJobConfPath() { return datum.jobConfPath.toString(); }
+  public String getJobConfPath() { return datum.getJobConfPath().toString(); }
   /** Get the acls configured for the job **/
   public Map<JobACL, AccessControlList> getJobAcls() {
     Map<JobACL, AccessControlList> jobAcls =
         new HashMap<JobACL, AccessControlList>();
     for (JobACL jobACL : JobACL.values()) {
       Utf8 jobACLsUtf8 = new Utf8(jobACL.getAclName());
-      if (datum.acls.containsKey(jobACLsUtf8)) {
-        jobAcls.put(jobACL, new AccessControlList(datum.acls.get(
+      if (datum.getAcls().containsKey(jobACLsUtf8)) {
+        jobAcls.put(jobACL, new AccessControlList(datum.getAcls().get(
             jobACLsUtf8).toString()));
       }
     }
@@ -170,36 +170,36 @@ public class JobSubmittedEvent implements HistoryEvent {
   }
   /** Get the id of the workflow */
   public String getWorkflowId() {
-    if (datum.workflowId != null) {
-      return datum.workflowId.toString();
+    if (datum.getWorkflowId() != null) {
+      return datum.getWorkflowId().toString();
     }
     return null;
   }
   /** Get the name of the workflow */
   public String getWorkflowName() {
-    if (datum.workflowName != null) {
-      return datum.workflowName.toString();
+    if (datum.getWorkflowName() != null) {
+      return datum.getWorkflowName().toString();
     }
     return null;
   }
   /** Get the node name of the workflow */
   public String getWorkflowNodeName() {
-    if (datum.workflowNodeName != null) {
-      return datum.workflowNodeName.toString();
+    if (datum.getWorkflowNodeName() != null) {
+      return datum.getWorkflowNodeName().toString();
     }
     return null;
   }
   /** Get the adjacencies of the workflow */
   public String getWorkflowAdjacencies() {
-    if (datum.workflowAdjacencies != null) {
-      return datum.workflowAdjacencies.toString();
+    if (datum.getWorkflowAdjacencies() != null) {
+      return datum.getWorkflowAdjacencies().toString();
     }
     return null;
   }
   /** Get the workflow tags */
   public String getWorkflowTags() {
-    if (datum.workflowTags != null) {
-      return datum.workflowTags.toString();
+    if (datum.getWorkflowTags() != null) {
+      return datum.getWorkflowTags().toString();
     }
     return null;
   }

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java

@@ -89,7 +89,9 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
   }
 
   /** Get the Job ID */
-  public JobID getJobId() { return JobID.forName(datum.jobid.toString()); }
+  public JobID getJobId() {
+    return JobID.forName(datum.getJobid().toString());
+  }
   /** Get the job finish time */
   public long getFinishTime() { return datum.getFinishTime(); }
   /** Get the number of finished maps */

+ 33 - 33
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java

@@ -125,48 +125,48 @@ public class MapAttemptFinishedEvent  implements HistoryEvent {
   public Object getDatum() {
     if (datum == null) {
       datum = new MapAttemptFinished();
-      datum.taskid = new Utf8(attemptId.getTaskID().toString());
-      datum.attemptId = new Utf8(attemptId.toString());
-      datum.taskType = new Utf8(taskType.name());
-      datum.taskStatus = new Utf8(taskStatus);
-      datum.mapFinishTime = mapFinishTime;
-      datum.finishTime = finishTime;
-      datum.hostname = new Utf8(hostname);
-      datum.port = port;
+      datum.setTaskid(new Utf8(attemptId.getTaskID().toString()));
+      datum.setAttemptId(new Utf8(attemptId.toString()));
+      datum.setTaskType(new Utf8(taskType.name()));
+      datum.setTaskStatus(new Utf8(taskStatus));
+      datum.setMapFinishTime(mapFinishTime);
+      datum.setFinishTime(finishTime);
+      datum.setHostname(new Utf8(hostname));
+      datum.setPort(port);
       if (rackName != null) {
-        datum.rackname = new Utf8(rackName);
+        datum.setRackname(new Utf8(rackName));
       }
-      datum.state = new Utf8(state);
-      datum.counters = EventWriter.toAvro(counters);
+      datum.setState(new Utf8(state));
+      datum.setCounters(EventWriter.toAvro(counters));
 
-      datum.clockSplits = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetWallclockTime(allSplits));
-      datum.cpuUsages = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetCPUTime(allSplits));
-      datum.vMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetVMemKbytes(allSplits));
-      datum.physMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetPhysMemKbytes(allSplits));
+      datum.setClockSplits(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetWallclockTime(allSplits)));
+      datum.setCpuUsages(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetCPUTime(allSplits)));
+      datum.setVMemKbytes(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetVMemKbytes(allSplits)));
+      datum.setPhysMemKbytes(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetPhysMemKbytes(allSplits)));
     }
     return datum;
   }
 
   public void setDatum(Object oDatum) {
     this.datum = (MapAttemptFinished)oDatum;
-    this.attemptId = TaskAttemptID.forName(datum.attemptId.toString());
-    this.taskType = TaskType.valueOf(datum.taskType.toString());
-    this.taskStatus = datum.taskStatus.toString();
-    this.mapFinishTime = datum.mapFinishTime;
-    this.finishTime = datum.finishTime;
-    this.hostname = datum.hostname.toString();
-    this.rackName = datum.rackname.toString();
-    this.port = datum.port;
-    this.state = datum.state.toString();
-    this.counters = EventReader.fromAvro(datum.counters);
-    this.clockSplits = AvroArrayUtils.fromAvro(datum.clockSplits);
-    this.cpuUsages = AvroArrayUtils.fromAvro(datum.cpuUsages);
-    this.vMemKbytes = AvroArrayUtils.fromAvro(datum.vMemKbytes);
-    this.physMemKbytes = AvroArrayUtils.fromAvro(datum.physMemKbytes);
+    this.attemptId = TaskAttemptID.forName(datum.getAttemptId().toString());
+    this.taskType = TaskType.valueOf(datum.getTaskType().toString());
+    this.taskStatus = datum.getTaskStatus().toString();
+    this.mapFinishTime = datum.getMapFinishTime();
+    this.finishTime = datum.getFinishTime();
+    this.hostname = datum.getHostname().toString();
+    this.rackName = datum.getRackname().toString();
+    this.port = datum.getPort();
+    this.state = datum.getState().toString();
+    this.counters = EventReader.fromAvro(datum.getCounters());
+    this.clockSplits = AvroArrayUtils.fromAvro(datum.getClockSplits());
+    this.cpuUsages = AvroArrayUtils.fromAvro(datum.getCpuUsages());
+    this.vMemKbytes = AvroArrayUtils.fromAvro(datum.getVMemKbytes());
+    this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes());
   }
 
   /** Get the task ID */

+ 35 - 35
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java

@@ -126,50 +126,50 @@ public class ReduceAttemptFinishedEvent  implements HistoryEvent {
   public Object getDatum() {
     if (datum == null) {
       datum = new ReduceAttemptFinished();
-      datum.taskid = new Utf8(attemptId.getTaskID().toString());
-      datum.attemptId = new Utf8(attemptId.toString());
-      datum.taskType = new Utf8(taskType.name());
-      datum.taskStatus = new Utf8(taskStatus);
-      datum.shuffleFinishTime = shuffleFinishTime;
-      datum.sortFinishTime = sortFinishTime;
-      datum.finishTime = finishTime;
-      datum.hostname = new Utf8(hostname);
-      datum.port = port;
+      datum.setTaskid(new Utf8(attemptId.getTaskID().toString()));
+      datum.setAttemptId(new Utf8(attemptId.toString()));
+      datum.setTaskType(new Utf8(taskType.name()));
+      datum.setTaskStatus(new Utf8(taskStatus));
+      datum.setShuffleFinishTime(shuffleFinishTime);
+      datum.setSortFinishTime(sortFinishTime);
+      datum.setFinishTime(finishTime);
+      datum.setHostname(new Utf8(hostname));
+      datum.setPort(port);
       if (rackName != null) {
-        datum.rackname = new Utf8(rackName);
+        datum.setRackname(new Utf8(rackName));
       }
-      datum.state = new Utf8(state);
-      datum.counters = EventWriter.toAvro(counters);
+      datum.setState(new Utf8(state));
+      datum.setCounters(EventWriter.toAvro(counters));
 
-      datum.clockSplits = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetWallclockTime(allSplits));
-      datum.cpuUsages = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetCPUTime(allSplits));
-      datum.vMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetVMemKbytes(allSplits));
-      datum.physMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
-        .arrayGetPhysMemKbytes(allSplits));
+      datum.setClockSplits(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetWallclockTime(allSplits)));
+      datum.setCpuUsages(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetCPUTime(allSplits)));
+      datum.setVMemKbytes(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetVMemKbytes(allSplits)));
+      datum.setPhysMemKbytes(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetPhysMemKbytes(allSplits)));
     }
     return datum;
   }
 
   public void setDatum(Object oDatum) {
     this.datum = (ReduceAttemptFinished)oDatum;
-    this.attemptId = TaskAttemptID.forName(datum.attemptId.toString());
-    this.taskType = TaskType.valueOf(datum.taskType.toString());
-    this.taskStatus = datum.taskStatus.toString();
-    this.shuffleFinishTime = datum.shuffleFinishTime;
-    this.sortFinishTime = datum.sortFinishTime;
-    this.finishTime = datum.finishTime;
-    this.hostname = datum.hostname.toString();
-    this.rackName = datum.rackname.toString();
-    this.port = datum.port;
-    this.state = datum.state.toString();
-    this.counters = EventReader.fromAvro(datum.counters);
-    this.clockSplits = AvroArrayUtils.fromAvro(datum.clockSplits);
-    this.cpuUsages = AvroArrayUtils.fromAvro(datum.cpuUsages);
-    this.vMemKbytes = AvroArrayUtils.fromAvro(datum.vMemKbytes);
-    this.physMemKbytes = AvroArrayUtils.fromAvro(datum.physMemKbytes);
+    this.attemptId = TaskAttemptID.forName(datum.getAttemptId().toString());
+    this.taskType = TaskType.valueOf(datum.getTaskType().toString());
+    this.taskStatus = datum.getTaskStatus().toString();
+    this.shuffleFinishTime = datum.getShuffleFinishTime();
+    this.sortFinishTime = datum.getSortFinishTime();
+    this.finishTime = datum.getFinishTime();
+    this.hostname = datum.getHostname().toString();
+    this.rackName = datum.getRackname().toString();
+    this.port = datum.getPort();
+    this.state = datum.getState().toString();
+    this.counters = EventReader.fromAvro(datum.getCounters());
+    this.clockSplits = AvroArrayUtils.fromAvro(datum.getClockSplits());
+    this.cpuUsages = AvroArrayUtils.fromAvro(datum.getCpuUsages());
+    this.vMemKbytes = AvroArrayUtils.fromAvro(datum.getVMemKbytes());
+    this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes());
   }
 
   /** Get the Task ID */

+ 17 - 17
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java

@@ -75,30 +75,30 @@ public class TaskAttemptFinishedEvent  implements HistoryEvent {
   public Object getDatum() {
     if (datum == null) {
       datum = new TaskAttemptFinished();
-      datum.taskid = new Utf8(attemptId.getTaskID().toString());
-      datum.attemptId = new Utf8(attemptId.toString());
-      datum.taskType = new Utf8(taskType.name());
-      datum.taskStatus = new Utf8(taskStatus);
-      datum.finishTime = finishTime;
+      datum.setTaskid(new Utf8(attemptId.getTaskID().toString()));
+      datum.setAttemptId(new Utf8(attemptId.toString()));
+      datum.setTaskType(new Utf8(taskType.name()));
+      datum.setTaskStatus(new Utf8(taskStatus));
+      datum.setFinishTime(finishTime);
       if (rackName != null) {
-        datum.rackname = new Utf8(rackName);
+        datum.setRackname(new Utf8(rackName));
       }
-      datum.hostname = new Utf8(hostname);
-      datum.state = new Utf8(state);
-      datum.counters = EventWriter.toAvro(counters);
+      datum.setHostname(new Utf8(hostname));
+      datum.setState(new Utf8(state));
+      datum.setCounters(EventWriter.toAvro(counters));
     }
     return datum;
   }
   public void setDatum(Object oDatum) {
     this.datum = (TaskAttemptFinished)oDatum;
-    this.attemptId = TaskAttemptID.forName(datum.attemptId.toString());
-    this.taskType = TaskType.valueOf(datum.taskType.toString());
-    this.taskStatus = datum.taskStatus.toString();
-    this.finishTime = datum.finishTime;
-    this.rackName = datum.rackname.toString();
-    this.hostname = datum.hostname.toString();
-    this.state = datum.state.toString();
-    this.counters = EventReader.fromAvro(datum.counters);
+    this.attemptId = TaskAttemptID.forName(datum.getAttemptId().toString());
+    this.taskType = TaskType.valueOf(datum.getTaskType().toString());
+    this.taskStatus = datum.getTaskStatus().toString();
+    this.finishTime = datum.getFinishTime();
+    this.rackName = datum.getRackname().toString();
+    this.hostname = datum.getHostname().toString();
+    this.state = datum.getState().toString();
+    this.counters = EventReader.fromAvro(datum.getCounters());
   }
 
   /** Get the task ID */

+ 26 - 23
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java

@@ -53,19 +53,19 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
       TaskType taskType, long startTime, String trackerName,
       int httpPort, int shufflePort, ContainerId containerId,
       String locality, String avataar) {
-    datum.attemptId = new Utf8(attemptId.toString());
-    datum.taskid = new Utf8(attemptId.getTaskID().toString());
-    datum.startTime = startTime;
-    datum.taskType = new Utf8(taskType.name());
-    datum.trackerName = new Utf8(trackerName);
-    datum.httpPort = httpPort;
-    datum.shufflePort = shufflePort;
-    datum.containerId = new Utf8(containerId.toString());
+    datum.setAttemptId(new Utf8(attemptId.toString()));
+    datum.setTaskid(new Utf8(attemptId.getTaskID().toString()));
+    datum.setStartTime(startTime);
+    datum.setTaskType(new Utf8(taskType.name()));
+    datum.setTrackerName(new Utf8(trackerName));
+    datum.setHttpPort(httpPort);
+    datum.setShufflePort(shufflePort);
+    datum.setContainerId(new Utf8(containerId.toString()));
     if (locality != null) {
-      datum.locality = new Utf8(locality);
+      datum.setLocality(new Utf8(locality));
     }
     if (avataar != null) {
-      datum.avataar = new Utf8(avataar);
+      datum.setAvataar(new Utf8(avataar));
     }
   }
 
@@ -75,7 +75,8 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
       long startTime, String trackerName, int httpPort, int shufflePort,
       String locality, String avataar) {
     this(attemptId, taskType, startTime, trackerName, httpPort, shufflePort,
-        ConverterUtils.toContainerId("container_-1_-1_-1_-1"), locality, avataar);
+        ConverterUtils.toContainerId("container_-1_-1_-1_-1"), locality,
+            avataar);
   }
 
   TaskAttemptStartedEvent() {}
@@ -86,22 +87,24 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
   }
 
   /** Get the task id */
-  public TaskID getTaskId() { return TaskID.forName(datum.taskid.toString()); }
+  public TaskID getTaskId() {
+    return TaskID.forName(datum.getTaskid().toString());
+  }
   /** Get the tracker name */
-  public String getTrackerName() { return datum.trackerName.toString(); }
+  public String getTrackerName() { return datum.getTrackerName().toString(); }
   /** Get the start time */
-  public long getStartTime() { return datum.startTime; }
+  public long getStartTime() { return datum.getStartTime(); }
   /** Get the task type */
   public TaskType getTaskType() {
-    return TaskType.valueOf(datum.taskType.toString());
+    return TaskType.valueOf(datum.getTaskType().toString());
   }
   /** Get the HTTP port */
-  public int getHttpPort() { return datum.httpPort; }
+  public int getHttpPort() { return datum.getHttpPort(); }
   /** Get the shuffle port */
-  public int getShufflePort() { return datum.shufflePort; }
+  public int getShufflePort() { return datum.getShufflePort(); }
   /** Get the attempt id */
   public TaskAttemptID getTaskAttemptId() {
-    return TaskAttemptID.forName(datum.attemptId.toString());
+    return TaskAttemptID.forName(datum.getAttemptId().toString());
   }
   /** Get the event type */
   public EventType getEventType() {
@@ -113,19 +116,19 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
   }
   /** Get the ContainerId */
   public ContainerId getContainerId() {
-    return ConverterUtils.toContainerId(datum.containerId.toString());
+    return ConverterUtils.toContainerId(datum.getContainerId().toString());
   }
   /** Get the locality */
   public String getLocality() {
-    if (datum.locality != null) {
-      return datum.locality.toString();
+    if (datum.getLocality() != null) {
+      return datum.getLocality().toString();
     }
     return null;
   }
   /** Get the avataar */
   public String getAvataar() {
-    if (datum.avataar != null) {
-      return datum.avataar.toString();
+    if (datum.getAvataar() != null) {
+      return datum.getAvataar().toString();
     }
     return null;
   }

+ 33 - 33
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java

@@ -133,28 +133,28 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   public Object getDatum() {
     if(datum == null) {
       datum = new TaskAttemptUnsuccessfulCompletion();
-      datum.taskid = new Utf8(attemptId.getTaskID().toString());
-      datum.taskType = new Utf8(taskType.name());
-      datum.attemptId = new Utf8(attemptId.toString());
-      datum.finishTime = finishTime;
-      datum.hostname = new Utf8(hostname);
+      datum.setTaskid(new Utf8(attemptId.getTaskID().toString()));
+      datum.setTaskType(new Utf8(taskType.name()));
+      datum.setAttemptId(new Utf8(attemptId.toString()));
+      datum.setFinishTime(finishTime);
+      datum.setHostname(new Utf8(hostname));
       if (rackName != null) {
-        datum.rackname = new Utf8(rackName);
+        datum.setRackname(new Utf8(rackName));
       }
-      datum.port = port;
-      datum.error = new Utf8(error);
-      datum.status = new Utf8(status);
-
-      datum.counters = EventWriter.toAvro(counters);
-
-      datum.clockSplits = AvroArrayUtils.toAvro(ProgressSplitsBlock
-          .arrayGetWallclockTime(allSplits));
-      datum.cpuUsages = AvroArrayUtils.toAvro(ProgressSplitsBlock
-          .arrayGetCPUTime(allSplits));
-      datum.vMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
-          .arrayGetVMemKbytes(allSplits));
-      datum.physMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
-          .arrayGetPhysMemKbytes(allSplits));
+      datum.setPort(port);
+      datum.setError(new Utf8(error));
+      datum.setStatus(new Utf8(status));
+
+      datum.setCounters(EventWriter.toAvro(counters));
+
+      datum.setClockSplits(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetWallclockTime(allSplits)));
+      datum.setCpuUsages(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetCPUTime(allSplits)));
+      datum.setVMemKbytes(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetVMemKbytes(allSplits)));
+      datum.setPhysMemKbytes(AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetPhysMemKbytes(allSplits)));
     }
     return datum;
   }
@@ -165,25 +165,25 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
     this.datum =
         (TaskAttemptUnsuccessfulCompletion)odatum;
     this.attemptId =
-        TaskAttemptID.forName(datum.attemptId.toString());
+        TaskAttemptID.forName(datum.getAttemptId().toString());
     this.taskType =
-        TaskType.valueOf(datum.taskType.toString());
-    this.finishTime = datum.finishTime;
-    this.hostname = datum.hostname.toString();
-    this.rackName = datum.rackname.toString();
-    this.port = datum.port;
-    this.status = datum.status.toString();
-    this.error = datum.error.toString();
+        TaskType.valueOf(datum.getTaskType().toString());
+    this.finishTime = datum.getFinishTime();
+    this.hostname = datum.getHostname().toString();
+    this.rackName = datum.getRackname().toString();
+    this.port = datum.getPort();
+    this.status = datum.getStatus().toString();
+    this.error = datum.getError().toString();
     this.counters =
-        EventReader.fromAvro(datum.counters);
+        EventReader.fromAvro(datum.getCounters());
     this.clockSplits =
-        AvroArrayUtils.fromAvro(datum.clockSplits);
+        AvroArrayUtils.fromAvro(datum.getClockSplits());
     this.cpuUsages =
-        AvroArrayUtils.fromAvro(datum.cpuUsages);
+        AvroArrayUtils.fromAvro(datum.getCpuUsages());
     this.vMemKbytes =
-        AvroArrayUtils.fromAvro(datum.vMemKbytes);
+        AvroArrayUtils.fromAvro(datum.getVMemKbytes());
     this.physMemKbytes =
-        AvroArrayUtils.fromAvro(datum.physMemKbytes);
+        AvroArrayUtils.fromAvro(datum.getPhysMemKbytes());
   }
 
   /** Get the task id */

+ 16 - 16
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java

@@ -80,16 +80,16 @@ public class TaskFailedEvent implements HistoryEvent {
   public Object getDatum() {
     if(datum == null) {
       datum = new TaskFailed();
-      datum.taskid = new Utf8(id.toString());
-      datum.error = new Utf8(error);
-      datum.finishTime = finishTime;
-      datum.taskType = new Utf8(taskType.name());
-      datum.failedDueToAttempt =
+      datum.setTaskid(new Utf8(id.toString()));
+      datum.setError(new Utf8(error));
+      datum.setFinishTime(finishTime);
+      datum.setTaskType(new Utf8(taskType.name()));
+      datum.setFailedDueToAttempt(
           failedDueToAttempt == null
           ? null
-          : new Utf8(failedDueToAttempt.toString());
-      datum.status = new Utf8(status);
-      datum.counters = EventWriter.toAvro(counters);
+          : new Utf8(failedDueToAttempt.toString()));
+      datum.setStatus(new Utf8(status));
+      datum.setCounters(EventWriter.toAvro(counters));
     }
     return datum;
   }
@@ -97,19 +97,19 @@ public class TaskFailedEvent implements HistoryEvent {
   public void setDatum(Object odatum) {
     this.datum = (TaskFailed)odatum;
     this.id =
-        TaskID.forName(datum.taskid.toString());
+        TaskID.forName(datum.getTaskid().toString());
     this.taskType =
-        TaskType.valueOf(datum.taskType.toString());
-    this.finishTime = datum.finishTime;
-    this.error = datum.error.toString();
+        TaskType.valueOf(datum.getTaskType().toString());
+    this.finishTime = datum.getFinishTime();
+    this.error = datum.getError().toString();
     this.failedDueToAttempt =
-        datum.failedDueToAttempt == null
+        datum.getFailedDueToAttempt() == null
         ? null
         : TaskAttemptID.forName(
-            datum.failedDueToAttempt.toString());
-    this.status = datum.status.toString();
+            datum.getFailedDueToAttempt().toString());
+    this.status = datum.getStatus().toString();
     this.counters =
-        EventReader.fromAvro(datum.counters);
+        EventReader.fromAvro(datum.getCounters());
   }
 
   /** Get the task id */

+ 13 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java

@@ -68,30 +68,30 @@ public class TaskFinishedEvent implements HistoryEvent {
   public Object getDatum() {
     if (datum == null) {
       datum = new TaskFinished();
-      datum.taskid = new Utf8(taskid.toString());
+      datum.setTaskid(new Utf8(taskid.toString()));
       if(successfulAttemptId != null)
       {
-        datum.successfulAttemptId = new Utf8(successfulAttemptId.toString());
+        datum.setSuccessfulAttemptId(new Utf8(successfulAttemptId.toString()));
       }
-      datum.finishTime = finishTime;
-      datum.counters = EventWriter.toAvro(counters);
-      datum.taskType = new Utf8(taskType.name());
-      datum.status = new Utf8(status);
+      datum.setFinishTime(finishTime);
+      datum.setCounters(EventWriter.toAvro(counters));
+      datum.setTaskType(new Utf8(taskType.name()));
+      datum.setStatus(new Utf8(status));
     }
     return datum;
   }
 
   public void setDatum(Object oDatum) {
     this.datum = (TaskFinished)oDatum;
-    this.taskid = TaskID.forName(datum.taskid.toString());
-    if (datum.successfulAttemptId != null) {
+    this.taskid = TaskID.forName(datum.getTaskid().toString());
+    if (datum.getSuccessfulAttemptId() != null) {
       this.successfulAttemptId = TaskAttemptID
-          .forName(datum.successfulAttemptId.toString());
+          .forName(datum.getSuccessfulAttemptId().toString());
     }
-    this.finishTime = datum.finishTime;
-    this.taskType = TaskType.valueOf(datum.taskType.toString());
-    this.status = datum.status.toString();
-    this.counters = EventReader.fromAvro(datum.counters);
+    this.finishTime = datum.getFinishTime();
+    this.taskType = TaskType.valueOf(datum.getTaskType().toString());
+    this.status = datum.getStatus().toString();
+    this.counters = EventReader.fromAvro(datum.getCounters());
   }
 
   /** Get task id */

+ 12 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java

@@ -42,10 +42,10 @@ public class TaskStartedEvent implements HistoryEvent {
    */
   public TaskStartedEvent(TaskID id, long startTime, 
       TaskType taskType, String splitLocations) {
-    datum.taskid = new Utf8(id.toString());
-    datum.splitLocations = new Utf8(splitLocations);
-    datum.startTime = startTime;
-    datum.taskType = new Utf8(taskType.name());
+    datum.setTaskid(new Utf8(id.toString()));
+    datum.setSplitLocations(new Utf8(splitLocations));
+    datum.setStartTime(startTime);
+    datum.setTaskType(new Utf8(taskType.name()));
   }
 
   TaskStartedEvent() {}
@@ -54,14 +54,18 @@ public class TaskStartedEvent implements HistoryEvent {
   public void setDatum(Object datum) { this.datum = (TaskStarted)datum; }
 
   /** Get the task id */
-  public TaskID getTaskId() { return TaskID.forName(datum.taskid.toString()); }
+  public TaskID getTaskId() {
+    return TaskID.forName(datum.getTaskid().toString());
+  }
   /** Get the split locations, applicable for map tasks */
-  public String getSplitLocations() { return datum.splitLocations.toString(); }
+  public String getSplitLocations() {
+    return datum.getSplitLocations().toString();
+  }
   /** Get the start time of the task */
-  public long getStartTime() { return datum.startTime; }
+  public long getStartTime() { return datum.getStartTime(); }
   /** Get the task type */
   public TaskType getTaskType() {
-    return TaskType.valueOf(datum.taskType.toString());
+    return TaskType.valueOf(datum.getTaskType().toString());
   }
   /** Get the event type */
   public EventType getEventType() {

+ 6 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java

@@ -41,8 +41,8 @@ public class TaskUpdatedEvent implements HistoryEvent {
    * @param finishTime Finish time of the task
    */
   public TaskUpdatedEvent(TaskID id, long finishTime) {
-    datum.taskid = new Utf8(id.toString());
-    datum.finishTime = finishTime;
+    datum.setTaskid(new Utf8(id.toString()));
+    datum.setFinishTime(finishTime);
   }
 
   TaskUpdatedEvent() {}
@@ -51,9 +51,11 @@ public class TaskUpdatedEvent implements HistoryEvent {
   public void setDatum(Object datum) { this.datum = (TaskUpdated)datum; }
 
   /** Get the task ID */
-  public TaskID getTaskId() { return TaskID.forName(datum.taskid.toString()); }
+  public TaskID getTaskId() {
+    return TaskID.forName(datum.getTaskid().toString());
+  }
   /** Get the task finish time */
-  public long getFinishTime() { return datum.finishTime; }
+  public long getFinishTime() { return datum.getFinishTime(); }
   /** Get the event type */
   public EventType getEventType() {
     return EventType.TASK_UPDATED;