Uploaded image for project: 'Hadoop Map/Reduce'
  1. Hadoop Map/Reduce
  2. MAPREDUCE-4329

security.task.umbilical.protocol.acl should not be configurable

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 1.0.3
    • None
    • security
    • None

    Description

      On running MapReduce job, username is changed to jobid and the job fails.
      Exception is as follows:

      2012-06-08 19:39:26,555 WARN org.apache.hadoop.security.ShellBasedUnixGroupsMapping: got exception trying to get groups for user job_201206081934_0002
      org.apache.hadoop.util.Shell$ExitCodeException: id: job_201206081934_0002: no such user
      
              at org.apache.hadoop.util.Shell.runCommand(Shell.java:255)
              at org.apache.hadoop.util.Shell.run(Shell.java:182)
              at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:375)
              at org.apache.hadoop.util.Shell.execCommand(Shell.java:461)
              at org.apache.hadoop.util.Shell.execCommand(Shell.java:444)
              at org.apache.hadoop.security.ShellBasedUnixGroupsMapping.getUnixGroups(ShellBasedUnixGroupsMapping.java:68)
              at org.apache.hadoop.security.ShellBasedUnixGroupsMapping.getGroups(ShellBasedUnixGroupsMapping.java:45)
              at org.apache.hadoop.security.Groups.getGroups(Groups.java:79)
              at org.apache.hadoop.security.UserGroupInformation.getGroupNames(UserGroupInformation.java:1026)
              at org.apache.hadoop.security.authorize.AccessControlList.isUserAllowed(AccessControlList.java:141)
              at org.apache.hadoop.security.authorize.ServiceAuthorizationManager.authorize(ServiceAuthorizationManager.java:99)
              at org.apache.hadoop.ipc.Server.authorize(Server.java:1659)
              at org.apache.hadoop.ipc.Server$Connection.authorizeConnection(Server.java:1320)
              at org.apache.hadoop.ipc.Server$Connection.processOneRpc(Server.java:1286)
              at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1182)
              at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:537)
              at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:344)
              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:680)
      

      This issue can be reproduced by following steps:

      1. set hadoop.security.authorization = true in core-site.xml

        <property>
          <name>hadoop.security.authorization</name>
          <value>true</value>
        </property>
      

      2. set any value except for '*' to security.task.umbilical.protocol.acl in hadoop-policy.xml

        <property>
          <name>security.task.umbilical.protocol.acl</name>
          <value>sho sho</value>
          <description>ACL for TaskUmbilicalProtocol, used by the map and reduce 
          tasks to communicate with the parent tasktracker. 
          The ACL is a comma-separated list of user and group names. The user and 
          group list is separated by a blank. For e.g. "alice,bob users,wheel". 
          A special value of "*" means all users are allowed.</description>
        </property>
      

      3. run any mapreduce job.

      Code Analysis

      ./src/mapred/org/apache/hadoop/mapred/Child.java:102-118

          UserGroupInformation taskOwner 
           = UserGroupInformation.createRemoteUser(firstTaskid.getJobID().toString());
          taskOwner.addToken(jt);
          
          // Set the credentials
          defaultConf.setCredentials(credentials);
          
          final TaskUmbilicalProtocol umbilical = 
            taskOwner.doAs(new PrivilegedExceptionAction<TaskUmbilicalProtocol>() {
              @Override
              public TaskUmbilicalProtocol run() throws Exception {
                return (TaskUmbilicalProtocol)RPC.getProxy(TaskUmbilicalProtocol.class,
                    TaskUmbilicalProtocol.versionID,
                    address,
                    defaultConf);
              }
          });
      

      This code indicates that TaskUmbilicalProtocol uses jobid as username.
      This code came from MAPREDUCE-1457.
      https://issues.apache.org/jira/browse/MAPREDUCE-1457

      Devaraj said as follows in the JIRA:

      2) In Child.java, the task authenticates to the TaskTracker using the jobtoken. The username in the jobtoken is jobId. The doAs block done using taskOwner is required so that the username mentioned in the token and the one doing the operation matches.

      We can't change security.task.umbilical.protocol.acl and should always be '*' .
      TaskUmbilicalProtocol should be removed from MapReducePolicyProvider to disable security.task.umbilical.protocol.acl.

      Attachments

        1. MAPREDUCE-4329.txt
          5 kB
          Sho Shimauchi
        2. MAPREDUCE-4329.txt
          4 kB
          Sho Shimauchi

        Issue Links

          Activity

            People

              sho.shimauchi Sho Shimauchi
              sho.shimauchi Sho Shimauchi
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated: