You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-issues@hadoop.apache.org by "Harsh J Chouraria (JIRA)" <ji...@apache.org> on 2010/12/29 20:39:56 UTC

[jira] Created: (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

No task may execute due to an Integer overflow possibility
----------------------------------------------------------

                 Key: MAPREDUCE-2236
                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
             Project: Hadoop Map/Reduce
          Issue Type: Bug
    Affects Versions: 0.20.2
         Environment: Linux, Hadoop 0.20.2
            Reporter: Harsh J Chouraria
            Assignee: Harsh J Chouraria
            Priority: Critical
             Fix For: 0.23.0


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:
{code}
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);
	}

}
{code}

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:
{code}
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'
{code}

The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
{code}
  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;
    }
{code}

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?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] [Resolved] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J (Resolved) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Harsh J resolved MAPREDUCE-2236.
--------------------------------

    Resolution: Not A Problem

>From above comments, apparently not a problem on 0.23+. We may reopen this if it is.
                
> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.24.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Arun C Murthy (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Arun C Murthy updated MAPREDUCE-2236:
-------------------------------------

    Status: Open  (was: Patch Available)

Sorry to come in late, the patch has gone stale. Can you please rebase? Thanks.

Given this is not an issue with MRv2 should we still commit this? I'm happy to, but not sure it's useful. Thanks.

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13047943#comment-13047943 ] 

Hadoop QA commented on MAPREDUCE-2236:
--------------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12482151/MAPREDUCE-2236.r2.diff
  against trunk revision 1134180.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 3 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    +1 findbugs.  The patch does not introduce any new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    -1 core tests.  The patch failed these core unit tests:
                  org.apache.hadoop.cli.TestMRCLI
                  org.apache.hadoop.conf.TestNoDefaultsJobConf
                  org.apache.hadoop.examples.terasort.TestTeraSort
                  org.apache.hadoop.fs.TestDFSIO
                  org.apache.hadoop.fs.TestFileSystem
                  org.apache.hadoop.ipc.TestSocketFactory
                  org.apache.hadoop.mapred.join.TestDatamerge
                  org.apache.hadoop.mapred.lib.TestDelegatingInputFormat
                  org.apache.hadoop.mapred.pipes.TestPipes
                  org.apache.hadoop.mapred.TestBadRecords
                  org.apache.hadoop.mapred.TestClusterMapReduceTestCase
                  org.apache.hadoop.mapred.TestCommandLineJobSubmission
                  org.apache.hadoop.mapred.TestCompressedEmptyMapOutputs
                  org.apache.hadoop.mapred.TestControlledMapReduceJob
                  org.apache.hadoop.mapred.TestFileInputFormat
                  org.apache.hadoop.mapred.TestJobClient
                  org.apache.hadoop.mapred.TestJobDirCleanup
                  org.apache.hadoop.mapred.TestJobHistory
                  org.apache.hadoop.mapred.TestJobName
                  org.apache.hadoop.mapred.TestJobQueueInformation
                  org.apache.hadoop.mapred.TestJobStatusPersistency
                  org.apache.hadoop.mapred.TestJobSysDirWithDFS
                  org.apache.hadoop.mapred.TestJobTrackerXmlJsp
                  org.apache.hadoop.mapred.TestLazyOutput
                  org.apache.hadoop.mapred.TestMapredHeartbeat
                  org.apache.hadoop.mapred.TestMapredSystemDir
                  org.apache.hadoop.mapred.TestMiniMRChildTask
                  org.apache.hadoop.mapred.TestMiniMRClasspath
                  org.apache.hadoop.mapred.TestMiniMRDFSCaching
                  org.apache.hadoop.mapred.TestMiniMRDFSSort
                  org.apache.hadoop.mapred.TestMiniMRWithDFSWithDistinctUsers
                  org.apache.hadoop.mapred.TestMiniMRWithDFS
                  org.apache.hadoop.mapred.TestMultipleLevelCaching
                  org.apache.hadoop.mapred.TestNodeRefresh
                  org.apache.hadoop.mapred.TestRecoveryManager
                  org.apache.hadoop.mapred.TestReduceFetchFromPartialMem
                  org.apache.hadoop.mapred.TestReduceFetch
                  org.apache.hadoop.mapred.TestSetupAndCleanupFailure
                  org.apache.hadoop.mapred.TestSeveral
                  org.apache.hadoop.mapred.TestSpecialCharactersInOutputPath
                  org.apache.hadoop.mapred.TestSubmitJob
                  org.apache.hadoop.mapred.TestTaskFail
                  org.apache.hadoop.mapred.TestWebUIAuthorization
                  org.apache.hadoop.mapreduce.lib.input.TestCombineFileInputFormat
                  org.apache.hadoop.mapreduce.lib.input.TestDelegatingInputFormat
                  org.apache.hadoop.mapreduce.lib.join.TestJoinDatamerge
                  org.apache.hadoop.mapreduce.lib.join.TestJoinProperties
                  org.apache.hadoop.mapreduce.security.TestBinaryTokenFile
                  org.apache.hadoop.mapreduce.security.TestTokenCacheOldApi
                  org.apache.hadoop.mapreduce.security.TestTokenCache
                  org.apache.hadoop.mapreduce.TestMapReduceLazyOutput
                  org.apache.hadoop.mapreduce.TestMRJobClient
                  org.apache.hadoop.security.authorize.TestServiceLevelAuthorization
                  org.apache.hadoop.security.TestMapredGroupMappingServiceRefresh
                  org.apache.hadoop.tools.TestCopyFiles
                  org.apache.hadoop.tools.TestDistCh
                  org.apache.hadoop.tools.TestHadoopArchives
                  org.apache.hadoop.tools.TestHarFileSystem

    -1 contrib tests.  The patch failed contrib unit tests.

    +1 system test framework.  The patch passed system test framework compile.

Test results: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/376//testReport/
Findbugs warnings: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/376//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/376//console

This message is automatically generated.

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13017910#comment-13017910 ] 

Hadoop QA commented on MAPREDUCE-2236:
--------------------------------------

+1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12475887/MAPREDUCE-2236.r1.diff
  against trunk revision 1090390.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 3 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    +1 findbugs.  The patch does not introduce any new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

    +1 system test framework.  The patch passed system test framework compile.

Test results: https://hudson.apache.org/hudson/job/PreCommit-MAPREDUCE-Build/162//testReport/
Findbugs warnings: https://hudson.apache.org/hudson/job/PreCommit-MAPREDUCE-Build/162//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: https://hudson.apache.org/hudson/job/PreCommit-MAPREDUCE-Build/162//console

This message is automatically generated.

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J Chouraria
>            Assignee: Harsh J Chouraria
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Harsh J updated MAPREDUCE-2236:
-------------------------------

    Attachment: MAPREDUCE-2236.r1.diff

Patch that caps it during set/load in {{TaskInProgress}}. Puts out a WARN log if the requested attempts exceed 100 (hardcoded, unconfigurable).

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] Commented: (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J Chouraria (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12975886#action_12975886 ] 

Harsh J Chouraria commented on MAPREDUCE-2236:
----------------------------------------------

We can also set a hard cap on the attempts amount when its being set; although how much should that be must be discussed. Integer.MAX_VALUE / 2 should do I guess?

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J Chouraria
>            Assignee: Harsh J Chouraria
>            Priority: Critical
>             Fix For: 0.23.0
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J Chouraria (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12978400#action_12978400 ] 

Harsh J Chouraria commented on MAPREDUCE-2236:
----------------------------------------------

100 sounds reasonable to me; would be a good cap for smaller clusters too.

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J Chouraria
>            Assignee: Harsh J Chouraria
>            Priority: Critical
>             Fix For: 0.23.0
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] [Updated] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J Chouraria (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Harsh J Chouraria updated MAPREDUCE-2236:
-----------------------------------------

    Attachment: MAPREDUCE-2236.r1.diff

Patch that caps maximum attempts at 100.

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J Chouraria
>            Assignee: Harsh J Chouraria
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] Commented: (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J Chouraria (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13007091#comment-13007091 ] 

Harsh J Chouraria commented on MAPREDUCE-2236:
----------------------------------------------

I'm wondering on how to cap this? Would it be best capped at the set level, or checked and capped at the get level?

I'm thinking 'get' is better.

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J Chouraria
>            Assignee: Harsh J Chouraria
>            Priority: Critical
>             Fix For: 0.23.0
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] Commented: (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Arun C Murthy (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12977160#action_12977160 ] 

Arun C Murthy commented on MAPREDUCE-2236:
------------------------------------------

A hard-cap should be on the order of 100 or so... 100 attempts per task is nearly 1 attempt on every rack in a 4k-node cluster. Thoughts?

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J Chouraria
>            Assignee: Harsh J Chouraria
>            Priority: Critical
>             Fix For: 0.23.0
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] [Updated] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J Chouraria (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Harsh J Chouraria updated MAPREDUCE-2236:
-----------------------------------------

    Status: Patch Available  (was: Open)

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J Chouraria
>            Assignee: Harsh J Chouraria
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Harsh J updated MAPREDUCE-2236:
-------------------------------

    Attachment: MAPREDUCE-2236.r2.diff

Rebased patch, should apply cleanly to current trunk.

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Harsh J updated MAPREDUCE-2236:
-------------------------------

    Release Note: Cap maximum retry attempts to 100, to avoid an overflow possibility.  (was: No task may run due to an Integer overflow possibility while setting the maximum attempts for a task.)

(Change release note.)

This patch would change an existing possibility by limiting it down, would it be considered as an "incompatible" change due to this?

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13070784#comment-13070784 ] 

Todd Lipcon commented on MAPREDUCE-2236:
----------------------------------------

I think it's better to cap it in TaskInProgress where it's pulled out of the conf -- that's more consistent with what we do with other such "limits".

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (MAPREDUCE-2236) No task may execute due to an Integer overflow possibility

Posted by "Harsh J (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13098777#comment-13098777 ] 

Harsh J commented on MAPREDUCE-2236:
------------------------------------

Well do you want me to rebase or do you feel there's no need to? I'm not targeting anything lower than trunk here, so let me know if its relevant (I'll also like hows/whys :D)

> No task may execute due to an Integer overflow possibility
> ----------------------------------------------------------
>
>                 Key: MAPREDUCE-2236
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2236
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 0.20.2
>         Environment: Linux, Hadoop 0.20.2
>            Reporter: Harsh J
>            Assignee: Harsh J
>            Priority: Critical
>             Fix For: 0.23.0
>
>         Attachments: MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r1.diff, MAPREDUCE-2236.r2.diff
>
>
> 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:
> {code}
> 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);
> 	}
> }
> {code}
> 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:
> {code}
> 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'
> {code}
> The issue lies inside the TaskInProgress class (/o/a/h/mapred/TaskInProgress.java), at line 1018 (trunk), part of the getTaskToRun(String taskTracker) method.
> {code}
>   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;
>     }
> {code}
> 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?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira