Skip to content

Commit

Permalink
more avro related code that needs changes
Browse files Browse the repository at this point in the history
  • Loading branch information
pjfanning committed Feb 3, 2024
1 parent dbbc524 commit 7b0e062
Show file tree
Hide file tree
Showing 4 changed files with 20 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ public class JobBuilder {
private Map<ParsedHost, ParsedHost> allHosts =
new HashMap<ParsedHost, ParsedHost>();

private org.apache.hadoop.mapreduce.jobhistory.JhCounters EMPTY_COUNTERS =
private final static org.apache.hadoop.mapreduce.jobhistory.JhCounters EMPTY_COUNTERS =
new org.apache.hadoop.mapreduce.jobhistory.JhCounters();

/**
Expand Down Expand Up @@ -460,7 +460,7 @@ private void processTaskFinishedEvent(TaskFinishedEvent event) {
}
task.setFinishTime(event.getFinishTime());
task.setTaskStatus(getPre21Value(event.getTaskStatus()));
task.incorporateCounters(((TaskFinished) event.getDatum()).counters);
task.incorporateCounters(((TaskFinished) event.getDatum()).getCounters());
}

private void processTaskFailedEvent(TaskFailedEvent event) {
Expand All @@ -472,7 +472,7 @@ private void processTaskFailedEvent(TaskFailedEvent event) {
task.setFinishTime(event.getFinishTime());
task.setTaskStatus(getPre21Value(event.getTaskStatus()));
TaskFailed t = (TaskFailed)(event.getDatum());
task.putDiagnosticInfo(t.error.toString());
task.putDiagnosticInfo(t.getError().toString());
// killed task wouldn't have failed attempt.
if (t.getFailedDueToAttempt() != null) {
task.putFailedDueToAttemptId(t.getFailedDueToAttempt().toString());
Expand Down Expand Up @@ -542,7 +542,7 @@ private void processTaskAttemptFinishedEvent(TaskAttemptFinishedEvent event) {
}
attempt.setFinishTime(event.getFinishTime());
attempt
.incorporateCounters(((TaskAttemptFinished) event.getDatum()).counters);
.incorporateCounters(((TaskAttemptFinished) event.getDatum()).getCounters());
}

private void processReduceAttemptFinishedEvent(
Expand All @@ -568,7 +568,7 @@ private void processReduceAttemptFinishedEvent(
attempt.setShuffleFinished(event.getShuffleFinishTime());
attempt.setSortFinished(event.getSortFinishTime());
attempt
.incorporateCounters(((ReduceAttemptFinished) event.getDatum()).counters);
.incorporateCounters(((ReduceAttemptFinished) event.getDatum()).getCounters());
attempt.arraySetClockSplits(event.getClockSplits());
attempt.arraySetCpuUsages(event.getCpuUsages());
attempt.arraySetVMemKbytes(event.getVMemKbytes());
Expand Down Expand Up @@ -596,7 +596,7 @@ private void processMapAttemptFinishedEvent(MapAttemptFinishedEvent event) {
// is redundant, but making this will add future-proofing.
attempt.setFinishTime(event.getFinishTime());
attempt
.incorporateCounters(((MapAttemptFinished) event.getDatum()).counters);
.incorporateCounters(((MapAttemptFinished) event.getDatum()).getCounters());
attempt.arraySetClockSplits(event.getClockSplits());
attempt.arraySetCpuUsages(event.getCpuUsages());
attempt.arraySetVMemKbytes(event.getVMemKbytes());
Expand Down Expand Up @@ -661,11 +661,11 @@ private void processJobFinishedEvent(JobFinishedEvent event) {

JobFinished job = (JobFinished)event.getDatum();
Map<String, Long> countersMap =
JobHistoryUtils.extractCounters(job.totalCounters);
JobHistoryUtils.extractCounters(job.getTotalCounters());
result.putTotalCounters(countersMap);
countersMap = JobHistoryUtils.extractCounters(job.mapCounters);
countersMap = JobHistoryUtils.extractCounters(job.getMapCounters());
result.putMapCounters(countersMap);
countersMap = JobHistoryUtils.extractCounters(job.reduceCounters);
countersMap = JobHistoryUtils.extractCounters(job.getReduceCounters());
result.putReduceCounters(countersMap);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,9 +157,9 @@ static boolean isJobConfXml(String fileName) {
static Map<String, Long> extractCounters(JhCounters counters) {
Map<String, Long> countersMap = new HashMap<String, Long>();
if (counters != null) {
for (JhCounterGroup group : counters.groups) {
for (JhCounter counter : group.counts) {
countersMap.put(counter.name.toString(), counter.value);
for (JhCounterGroup group : counters.getGroups()) {
for (JhCounter counter : group.getCounts()) {
countersMap.put(counter.getName().toString(), counter.getValue());
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -268,11 +268,11 @@ private static void incorporateCounter(SetField thunk, JhCounters counters,
String counterName) {
counterName = canonicalizeCounterName(counterName);

for (JhCounterGroup group : counters.groups) {
for (JhCounter counter : group.counts) {
for (JhCounterGroup group : counters.getGroups()) {
for (JhCounter counter : group.getCounts()) {
if (counterName
.equals(canonicalizeCounterName(counter.name.toString()))) {
thunk.set(counter.value);
.equals(canonicalizeCounterName(counter.getName().toString()))) {
thunk.set(counter.getValue());
return;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -636,11 +636,11 @@ private static void incorporateCounter(SetField thunk, JhCounters counters,
String counterName) {
counterName = canonicalizeCounterName(counterName);

for (JhCounterGroup group : counters.groups) {
for (JhCounter counter : group.counts) {
for (JhCounterGroup group : counters.getGroups()) {
for (JhCounter counter : group.getCounts()) {
if (counterName
.equals(canonicalizeCounterName(counter.name.toString()))) {
thunk.set(counter.value);
.equals(canonicalizeCounterName(counter.getName().toString()))) {
thunk.set(counter.getValue());
return;
}
}
Expand Down

0 comments on commit 7b0e062

Please sign in to comment.