[
https://issues.apache.org/jira/browse/HBASE-5498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13291104#comment-13291104
]
Jonathan Hsieh commented on HBASE-5498:
---------------------------------------
This is a little bit outside my wheelhouse so expect I'd expect more feedback
from Andrew or Todd (or me once I ingest more of this). Here are some quick
comments / questions from a skim of the patch:
- Is the design doc on HCatalog wiki relevant to the new implementation? (Why
in HCatalog?)
- We should also probably add a postBulkLoadHFile to be symmetric. If this
drags out, consider adding a "add postBulkLoadHFile and preBulkLoadHFile to
RegionObserver/BaseRegionObserver" subissue -- it would likely committed
quickly alone.
- This needs null check guard on getCoprocessHost()
{code}
}
+ region.getCoprocessorHost()
+ .preBulkLoadHFile(familyPaths);
boolean loaded = region.bulkLoadHFiles(familyPaths);
{code}
- SecureBulkLoadEndPoint/SecureBulkLoadProtocol will need an apache license
header
> 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
> 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