Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Not A Problem
-
0.20.2
-
None
-
None
-
None
-
Linux, Hadoop 0.20.2
-
Cap maximum retry attempts to 100, to avoid an overflow possibility.
-
Integer overflow, TaskInProgress
Description
If the attempts is configured to use Integer.MAX_VALUE, an overflow occurs inside TaskInProgress, and thereby no task is attempted by the cluster and the map tasks stay in pending state forever.
For example, here's a job driver that causes this:
import java.io.IOException; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.NullOutputFormat; @SuppressWarnings("deprecation") public class IntegerOverflow { /** * @param args * @throws IOException */ @SuppressWarnings("deprecation") public static void main(String[] args) throws IOException { JobConf conf = new JobConf(); Path inputPath = new Path("ignore"); FileSystem fs = FileSystem.get(conf); if (!fs.exists(inputPath)) { FSDataOutputStream out = fs.create(inputPath); out.writeChars("Test"); out.close(); } conf.setInputFormat(TextInputFormat.class); conf.setOutputFormat(NullOutputFormat.class); FileInputFormat.addInputPath(conf, inputPath); conf.setMapperClass(IdentityMapper.class); conf.setNumMapTasks(1); // Problem inducing line follows. conf.setMaxMapAttempts(Integer.MAX_VALUE); // No reducer in this test, although setMaxReduceAttempts leads to the same problem. conf.setNumReduceTasks(0); JobClient.runJob(conf); } }
The above code will not let any map task run. Additionally, a log would be created inside JobTracker logs with the following information that clearly shows the overflow:
2010-12-30 00:59:07,836 WARN org.apache.hadoop.mapred.TaskInProgress: Exceeded limit of -2147483648 (plus 0 killed) attempts for the tip 'task_201012300058_0001_m_000000'
The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
public Task getTaskToRun(String taskTracker) throws IOException { // Create the 'taskid'; do not count the 'killed' tasks against the job! TaskAttemptID taskid = null; /* ============ THIS LINE v ====================================== */ if (nextTaskId < (MAX_TASK_EXECS + maxTaskAttempts + numKilledTasks)) { /* ============ THIS LINE ^====================================== */ // Make sure that the attempts are unqiue across restarts int attemptId = job.getNumRestarts() * NUM_ATTEMPTS_PER_RESTART + nextTaskId; taskid = new TaskAttemptID( id, attemptId); ++nextTaskId; } else { LOG.warn("Exceeded limit of " + (MAX_TASK_EXECS + maxTaskAttempts) + " (plus " + numKilledTasks + " killed)" + " attempts for the tip '" + getTIPId() + "'"); return null; }
Since all three variables being added are integer in type, one of them being Integer.MAX_VALUE makes the condition fail with an overflow, thereby logging and returning a null as the result is negative.
One solution would be to make one of these variables into a long, so the addition does not overflow?