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

Alejandro Abdelnur commented on HDFS-6826:
------------------------------------------

[~daryn], thanks for the detailed review. A few comments on it.

On #2, I don’t see how this would happen unless you are adding a new 
authorization property or a new way of checking permissions independent of the 
current one.

On #4, my bad, I’ve missed the special entry point of webhdfs, that is easily 
fixable.

ON #5, AFAIK the performance of thread locals has improved significantly since 
its inception. I don’t think this should be an issue.

On #8,the {{DefaultAuthorizationProvider}} is the implementation provide by 
Hadoop. Also, its implementation is not ‘public’.

On #9, I think is a reasonable assumption for now to have a single authz 
provider. If more than one, that could be done with a multiplexor 
implementation (as you could assume they will have zero intersection on the 
HDFS sub-trees they manage.

Also, I think that because v7.6 works on resolved INodes, we don’t need to 
worry about snapshots paths or inodes paths.


> Plugin interface to enable delegation of HDFS authorization assertions
> ----------------------------------------------------------------------
>
>                 Key: HDFS-6826
>                 URL: https://issues.apache.org/jira/browse/HDFS-6826
>             Project: Hadoop HDFS
>          Issue Type: New Feature
>          Components: security
>    Affects Versions: 2.4.1
>            Reporter: Alejandro Abdelnur
>            Assignee: Alejandro Abdelnur
>         Attachments: HDFS-6826-idea.patch, HDFS-6826-idea2.patch, 
> HDFS-6826-permchecker.patch, HDFS-6826v3.patch, HDFS-6826v4.patch, 
> HDFS-6826v5.patch, HDFS-6826v6.patch, HDFS-6826v7.1.patch, 
> HDFS-6826v7.2.patch, HDFS-6826v7.3.patch, HDFS-6826v7.4.patch, 
> HDFS-6826v7.5.patch, HDFS-6826v7.6.patch, HDFS-6826v7.patch, 
> HDFS-6826v8.patch, HDFS-6826v9.patch, 
> HDFSPluggableAuthorizationProposal-v2.pdf, 
> HDFSPluggableAuthorizationProposal.pdf
>
>
> When Hbase data, HiveMetaStore data or Search data is accessed via services 
> (Hbase region servers, HiveServer2, Impala, Solr) the services can enforce 
> permissions on corresponding entities (databases, tables, views, columns, 
> search collections, documents). It is desirable, when the data is accessed 
> directly by users accessing the underlying data files (i.e. from a MapReduce 
> job), that the permission of the data files map to the permissions of the 
> corresponding data entity (i.e. table, column family or search collection).
> To enable this we need to have the necessary hooks in place in the NameNode 
> to delegate authorization to an external system that can map HDFS 
> files/directories to data entities and resolve their permissions based on the 
> data entities permissions.
> I’ll be posting a design proposal in the next few days.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to