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

Benoy Antony commented on HDFS-6134:
------------------------------------

[~tucu00]

In the selected option (option 3), the HDFS (NN to be exact) fetches the keys. 
As long as the client is authenticated and authorized by HDFS, encrypted data 
can be read. 
The process is equivalent to a party trying to get into a house.
Party is validated by security person (NN) based on party's identity card  ( 
auth token) and security person's list of authorized persons ( acls on the 
file/directory).  
If the party passes these checks, security person hands the key to the party. 
(Option 4 is slightly different, security person doesn't hand over the key to 
the party, but opens the house for the party.)
Also note that the security person(NN) needs to have access to keys of the all 
the houses.

The plus point is simplicity. But it sets a low hurdle for security breach. 

A malicious party can get access to the keys  by impersonating an authorized 
party. In the HDFS case, this is possible by 

1) Stealing a TGT/delegation token
2) Admin impersonating a user by giving himself ability to impersonate.

Now if the same key  can decrypt other files, then malicious party can decrypt 
other files without going through further authn/authz checks. Option 4 doesn't 
have this specific vulnerability since the keys are handed over to the client. 

In some cases , this risk is acceptable.  

But in some cases , it is not acceptable. More protection is needed by 
requiring that client has to obtain the key himself. 

In the house example, Party is validated by security person based on party's 
identity card  ( auth token) and security person's  list of authorized persons 
( acls on the file/directory).  
If the party passes these checks, security person let's the party proceed to 
the house. But the party needs to have the key.
Thus even if one party can impersonate another party and thus fool the 
security, the impersonator cannot enter the house as he doesn't have the key.

This additional hurdle is one of the reasons for clients to obtain the key.
There could be other reasons. 

1) In some cases, another entity (NN) cannot have access to keys which it 
doesn't own.
2) In some cases, NN may be in a location which has no connectivity to key 
store.

So my question.

Is it possible to make the key provisioning part customizable so that depending 
upon the requirement, the key can be obtained by client themselves or obtained 
by the HDFS ? 
(support option 2 and option 3)
If so , it may also make sense to have both options supported on the same 
cluster instance as the level of security vary based on data.

> Transparent data at rest encryption
> -----------------------------------
>
>                 Key: HDFS-6134
>                 URL: https://issues.apache.org/jira/browse/HDFS-6134
>             Project: Hadoop HDFS
>          Issue Type: New Feature
>          Components: security
>    Affects Versions: 2.3.0
>            Reporter: Alejandro Abdelnur
>            Assignee: Alejandro Abdelnur
>         Attachments: HDFSDataAtRestEncryption.pdf
>
>
> Because of privacy and security regulations, for many industries, sensitive 
> data at rest must be in encrypted form. For example: the health­care industry 
> (HIPAA regulations), the card payment industry (PCI DSS regulations) or the 
> US government (FISMA regulations).
> This JIRA aims to provide a mechanism to encrypt HDFS data at rest that can 
> be used transparently by any application accessing HDFS via Hadoop Filesystem 
> Java API, Hadoop libhdfs C library, or WebHDFS REST API.
> The resulting implementation should be able to be used in compliance with 
> different regulation requirements.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to