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 "Luke Lu (JIRA)" <ji...@apache.org> on 2012/07/05 12:15:35 UTC

[jira] [Created] (MAPREDUCE-4398) Fix mapred.system.dir permission error with FairScheduler

Luke Lu created MAPREDUCE-4398:
----------------------------------

             Summary: Fix mapred.system.dir permission error with FairScheduler
                 Key: MAPREDUCE-4398
                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4398
             Project: Hadoop Map/Reduce
          Issue Type: Bug
          Components: contrib/fair-share
    Affects Versions: 1.0.3
            Reporter: Luke Lu
            Assignee: Yu Gao


Incorrect job initialization logic in FairScheduler causes mysterious intermittent mapred.system.dir permission errors.

--
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] [Commented] (MAPREDUCE-4398) Fix mapred.system.dir permission error with FairScheduler

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

Luke Lu commented on MAPREDUCE-4398:
------------------------------------

The "magic" number of 4 is the default number of job init threads (mapred.jobinit.threads). You have to submit 4 (or precisely mapred.jobinit.threads) or more jobs as the jobtracker user at the same time to make sure the job init thread are initialized as the system user so they can access the mapred.system.dir (for security reasons, it must be 700). Otherwise, some of the job init threads will be initialized as whatever user who first submits a job. This can lead to seemingly more bizarre behavior: some time it works (the job is initialized by one of the system threads) and sometime it doesn't (the job is initialized by one of the user threads). Once you know the root cause, it's pretty trivial to come up with a patch. The default fifo scheduler and capacity scheduler do not have this bug.
                
> Fix mapred.system.dir permission error with FairScheduler
> ---------------------------------------------------------
>
>                 Key: MAPREDUCE-4398
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4398
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: contrib/fair-share
>    Affects Versions: 1.0.3
>            Reporter: Luke Lu
>            Assignee: Yu Gao
>
> Incorrect job initialization logic in FairScheduler causes mysterious intermittent mapred.system.dir permission errors.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (MAPREDUCE-4398) Fix mapred.system.dir permission error with FairScheduler

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

Arpit Gupta commented on MAPREDUCE-4398:
----------------------------------------

The following stack trace was seen when using fair scheduler with 1.0.3 release

{code}
Generating 10000000000 using 2 maps with step of 5000000000
12/10/09 19:04:09 INFO mapred.JobClient: Running job: job_201210091900_0002
12/10/09 19:04:10 INFO mapred.JobClient:  map 0% reduce 0%
12/10/09 19:04:10 INFO mapred.JobClient: Job complete: job_201210091900_0002
12/10/09 19:04:10 INFO mapred.JobClient: Counters: 0
12/10/09 19:04:10 INFO mapred.JobClient: Job Failed: Job initialization failed:
org.apache.hadoop.security.AccessControlException: org.apache.hadoop.security.AccessControlException: Permission denied: user=robing, access=EXECUTE, inode="system":mapred:hadoop:rwx------
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:95)
at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:57)
at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.<init>(DFSClient.java:3251)
at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:713)
at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:182)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:555)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:536)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:443)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:435)
at org.apache.hadoop.security.Credentials.writeTokenStorageFile(Credentials.java:169)
at org.apache.hadoop.mapred.JobInProgress.generateAndStoreTokens(JobInProgress.java:3537)
at org.apache.hadoop.mapred.JobInProgress.initTasks(JobInProgress.java:696)
at org.apache.hadoop.mapred.JobTracker.initJob(JobTracker.java:4207)
at org.apache.hadoop.mapred.FairScheduler$JobInitializer$InitJob.run(FairScheduler.java:291)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)
{code}

It looks like when using fair scheduler the job token file is being written (in the mapred.system.dir) as the user running the job where as if we use the default scheduler that file is being written as the user running mr.
                
> Fix mapred.system.dir permission error with FairScheduler
> ---------------------------------------------------------
>
>                 Key: MAPREDUCE-4398
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4398
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: contrib/fair-share
>    Affects Versions: 1.0.3
>            Reporter: Luke Lu
>            Assignee: Yu Gao
>
> Incorrect job initialization logic in FairScheduler causes mysterious intermittent mapred.system.dir permission errors.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Resolved] (MAPREDUCE-4398) Fix mapred.system.dir permission error with FairScheduler

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

Luke Lu resolved MAPREDUCE-4398.
--------------------------------

       Resolution: Duplicate
    Fix Version/s: 1.2.0

The changes in MAPREDUCE-4451 will resolve this issue as well, whether security is enabled or not.
                
> Fix mapred.system.dir permission error with FairScheduler
> ---------------------------------------------------------
>
>                 Key: MAPREDUCE-4398
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4398
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: contrib/fair-share
>    Affects Versions: 1.0.3
>            Reporter: Luke Lu
>            Assignee: Yu Gao
>             Fix For: 1.2.0
>
>
> Incorrect job initialization logic in FairScheduler causes mysterious intermittent mapred.system.dir permission errors.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (MAPREDUCE-4398) Fix mapred.system.dir permission error with FairScheduler

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

Luke Lu commented on MAPREDUCE-4398:
------------------------------------

A workaround is to submit 4 or more jobs as the user of the jobtracker and everything will work fine afterwards. BTW, IBM BigInsights community version (open source) contains the right fix (properly initialize job init threads) to fairscheduler since BigInsights 1.3.0.1. Unfortunately IBM devs are too busy to port/submit the patches to Apache right now :)
                
> Fix mapred.system.dir permission error with FairScheduler
> ---------------------------------------------------------
>
>                 Key: MAPREDUCE-4398
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4398
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: contrib/fair-share
>    Affects Versions: 1.0.3
>            Reporter: Luke Lu
>            Assignee: Yu Gao
>
> Incorrect job initialization logic in FairScheduler causes mysterious intermittent mapred.system.dir permission errors.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira