hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Laxman (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-5498) Secure Bulk Load
Date Wed, 11 Jul 2012 14:12:36 GMT

    [ https://issues.apache.org/jira/browse/HBASE-5498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13411543#comment-13411543
] 

Laxman commented on HBASE-5498:
-------------------------------

I tried this draft patch. 

*Scenario*
1) As "hbase": Create a table "employee". Grant permissions to "test" user.
2) As "hdfs": Create dir "/test" and make "test" as owner.
3) As "test": Place the tsv input in "/test/input" and run importtsv from command line as
earlier.
4) As "test": Run the following snippet to do "completebulkload". I'm not sure whether this
is correct usage.
{code}
    HTable table = new HTable(HBaseConfiguration.create(), "employee");
    AuthenticationProtocol auth = table.coprocessorProxy(AuthenticationProtocol.class,
        HConstants.EMPTY_START_ROW);
    Token<AuthenticationTokenIdentifier> token = auth.getAuthenticationToken();
    SecureBulkLoadProtocol proxy = table.coprocessorProxy(SecureBulkLoadProtocol.class,
        HConstants.EMPTY_START_ROW);
    proxy.secureCompleteBulkLoad("employee", "hdfs://10.18.40.92:9000/test/bulkload/output",
token);
{code}

First phase (i.e. importtsv) worked fine.
For the second phase (i.e. completebulkload) I used the following snippet as my client code.

But I'm hitting the following exception on region server side.
{noformat}
2012-07-11 19:23:38,529 ERROR org.apache.hadoop.hbase.security.access.SecureBulkLoadEndPoint:
Failed to secure bulk load
org.apache.hadoop.security.AccessControlException: Permission denied
	at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkOwner(FSPermissionChecker.java:155)
	at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:147)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:4271)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkOwner(FSNamesystem.java:4227)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.setPermission(FSNamesystem.java:1010)
{noformat}
                
> 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

        

Mime
View raw message