[ https://issues.apache.org/jira/browse/HBASE-5498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13412719#comment-13412719 ]
Laxman commented on HBASE-5498: ------------------------------- Thanks for correction Francis. I tried as per your suggestion. Still no luck. Now we are hitting the same "Permission denied" problem. On further investigation, I found that the DFS requests(rename) is still going with "hbase" user. FS is initialized as "hbase" user during coprocessor initialization on startup. I moved FS initialization like below. {code} boolean success = ugi.doAs(new PrivilegedAction<Boolean>() { @Override public Boolean run() { FileSystem fs=null; try { fs=FileSystem.get(conf); setPermission(fs, srcPath, new FsPermission((short) 0777)); fs.rename(srcPath, contentPath); LOG.info("moving " + srcPath + " to " + contentPath); completeBulkLoad(tableName, contentPath); ... {code} With this, I'm hitting a brand new problem. {noformat} 2012-07-12 16:23:43,839 ERROR org.apache.hadoop.hbase.security.access.SecureBulkLoadEndPoint: Failed to complete bulk load java.lang.NullPointerException at javax.security.auth.Subject$ClassSet.populateSet(Subject.java:1351) at javax.security.auth.Subject$ClassSet.<init>(Subject.java:1317) at javax.security.auth.Subject.getPrivateCredentials(Subject.java:731) at org.apache.hadoop.security.UserGroupInformation.<init>(UserGroupInformation.java:488) at org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:514) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:346) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:327) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:121) {noformat} Based on this stacktrace analysis, this occurred as we continue without logging in "ugi.login()". But this ugi.login supports keytab/ticket cache which I don't have for "test" on region-server side. I think I'm missing some fundamental point in token delegation. Apologies for too much of text. > Secure Bulk Load > ---------------- > > Key: HBASE-5498 > URL: https://issues.apache.org/jira/browse/HBASE-5498 > Project: HBase > Issue Type: Improvement > Components: mapred, security > Reporter: Francis Liu > Assignee: Francis Liu > Fix For: 0.96.0 > > Attachments: HBASE-5498_draft.patch > > > Design doc: > https://cwiki.apache.org/confluence/display/HCATALOG/HBase+Secure+Bulk+Load > Short summary: > Security as it stands does not cover the bulkLoadHFiles() feature. Users > calling this method will bypass ACLs. Also loading is made more cumbersome in > a secure setting because of hdfs privileges. bulkLoadHFiles() moves the data > from user's directory to the hbase directory, which would require certain > write access privileges set. > Our solution is to create a coprocessor which makes use of AuthManager to > verify if a user has write access to the table. If so, launches a MR job as > the hbase user to do the importing (ie rewrite from text to hfiles). One > tricky part this job will have to do is impersonate the calling user when > reading the input files. We can do this by expecting the user to pass an hdfs > delegation token as part of the secureBulkLoad() coprocessor call and extend > an inputformat to make use of that token. The output is written to a > temporary directory accessible only by hbase and then bulkloadHFiles() is > called. -- 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