Skip to content

Commit

Permalink
Improve taskId detection in EsOutputFormat
Browse files Browse the repository at this point in the history
Relates #280

(cherry picked from commit 145ad76)
  • Loading branch information
costin committed Sep 26, 2014
1 parent 11e558b commit 21033ca
Show file tree
Hide file tree
Showing 3 changed files with 22 additions and 18 deletions.
Expand Up @@ -179,7 +179,7 @@ protected void init() throws IOException {
}

private int detectCurrentInstance(Configuration conf) {
TaskID taskID = TaskID.forName(HadoopCfgUtils.getTaskId(conf));
TaskID taskID = HadoopCfgUtils.getTaskID(conf);

if (taskID == null) {
log.warn(String.format("Cannot determine task id - redirecting writes in a random fashion"));
Expand Down
19 changes: 19 additions & 0 deletions mr/src/main/java/org/elasticsearch/hadoop/mr/HadoopCfgUtils.java
Expand Up @@ -23,6 +23,9 @@

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.TaskAttemptID;
import org.apache.hadoop.mapred.TaskID;
import org.elasticsearch.hadoop.util.StringUtils;
import org.elasticsearch.hadoop.util.unit.TimeValue;

/**
Expand Down Expand Up @@ -137,4 +140,20 @@ public static JobConf asJobConf(Configuration cfg) {
public static String getMapValueClass(Configuration cfg) {
return get(cfg, "mapred.mapoutput.value.class", "mapreduce.map.output.value.class");
}

public static TaskID getTaskID(Configuration cfg) {
// first try with the attempt since some Hadoop versions mix the two
String taskAttemptId = HadoopCfgUtils.getTaskAttemptId(cfg);
if (StringUtils.hasText(taskAttemptId)) {
return TaskAttemptID.forName(taskAttemptId).getTaskID();
}
else {
String taskIdProp = HadoopCfgUtils.getTaskId(cfg);
// double-check task id bug in Hadoop 2.5.x
if (StringUtils.hasText(taskIdProp) && !taskIdProp.contains("attempt")) {
return TaskID.forName(taskIdProp);
}
}
return null;
}
}
19 changes: 2 additions & 17 deletions mr/src/main/java/org/elasticsearch/hadoop/mr/HeartBeat.java
Expand Up @@ -24,11 +24,9 @@

import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.TaskAttemptID;
import org.apache.hadoop.mapred.TaskID;
import org.apache.hadoop.util.Progressable;
import org.elasticsearch.hadoop.util.Assert;
import org.elasticsearch.hadoop.util.StringUtils;
import org.elasticsearch.hadoop.util.unit.TimeValue;

/**
Expand All @@ -51,22 +49,9 @@ class HeartBeat {
this.rate = new TimeValue(tv.getMillis() - delay.getMillis(), TimeUnit.MILLISECONDS);
this.log = log;

TaskID taskID = null;
// first try with the attempt since some Hadoop versions mix the two

String taskAttemptId = HadoopCfgUtils.getTaskAttemptId(cfg);
if (StringUtils.hasText(taskAttemptId)) {
taskID = TaskAttemptID.forName(taskAttemptId).getTaskID();
}
else {
String taskIdProp = HadoopCfgUtils.getTaskId(cfg);
// double-check task id bug in Hadoop 2.5.x
if (StringUtils.hasText(taskIdProp) && !taskIdProp.contains("attempt")) {
taskID = TaskID.forName(taskIdProp);
}
}

String taskId;
TaskID taskID = HadoopCfgUtils.getTaskID(cfg);

if (taskID == null) {
log.warn("Cannot determine task id...");
taskId = "<unknown>";
Expand Down

0 comments on commit 21033ca

Please sign in to comment.