hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jing Zhao (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (HADOOP-10215) Cannot create hftp filesystem when using a proxy user ugi and a doAs on a secure cluster
Date Wed, 22 Jan 2014 19:53:20 GMT

     [ https://issues.apache.org/jira/browse/HADOOP-10215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Jing Zhao updated HADOOP-10215:
-------------------------------

    Attachment: HADOOP-10215.001.patch

Update the patch. The new patch makes sure that the DT is under the proxy user's name. 

I've tested the patch in my local security setup and the patch works fine. The testing code:
{code}
public class TestHftpFSWithProxyUser {
  public static void main(String[] argv) throws Exception {
    if (argv.length <= 1) {
      System.err.println("Usage: TestHftpFSWithProxyUser fs-uri proxyUser");
      return;
    }
    
    String fsUri = argv[0];
    String proxyUserName = argv[1];
    
    UserGroupInformation real = UserGroupInformation.getCurrentUser();
    System.out.println("Get real ugi: " + real.getShortUserName());
    
    UserGroupInformation proxy = UserGroupInformation.createProxyUser(
        proxyUserName, real);
    System.out.println("Create proxy ugi: " + proxy.getShortUserName());
    
    final Configuration conf = new Configuration();
    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsUri);
    
    FileStatus[] status = proxy.doAs(new PrivilegedExceptionAction<FileStatus[]>() {
      @Override
      public FileStatus[] run() throws Exception {
        FileSystem fs = FileSystem.get(conf);
        return fs.listStatus(new Path("/"));
      }
    });
    System.out.println("ls results: " + Arrays.asList(status).toString());
  }
}
{code}

> Cannot create hftp filesystem when using a proxy user ugi and a doAs on a secure cluster
> ----------------------------------------------------------------------------------------
>
>                 Key: HADOOP-10215
>                 URL: https://issues.apache.org/jira/browse/HADOOP-10215
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: security
>    Affects Versions: 2.2.0
>            Reporter: Arpit Gupta
>            Assignee: Jing Zhao
>         Attachments: HADOOP-10215.000.patch, HADOOP-10215.001.patch
>
>
> Noticed this while debugging issues in another application. We saw an error when trying
to do a FileSystem.get using an hftp file system on a secure cluster using a proxy user ugi.
> This is a small snippet used
> {code}
>  FileSystem testFS = ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
>             @Override
>             public FileSystem run() throws IOException {
>                 return FileSystem.get(hadoopConf);
>             }
>         });
> {code}
> The same code worked for hdfs and webhdfs but not for hftp when the ugi used was UserGroupInformation.createProxyUser



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Mime
View raw message