You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-dev@hadoop.apache.org by "Alejandro Abdelnur (JIRA)" <ji...@apache.org> on 2014/05/12 19:42:18 UTC

[jira] [Reopened] (HDFS-6134) Transparent data at rest encryption

     [ https://issues.apache.org/jira/browse/HDFS-6134?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alejandro Abdelnur reopened HDFS-6134:
--------------------------------------


[cross-posting with HADOOP-10150]

Reopening HDFS-6134

After some offline discussions with Yi, Tianyou, ATM, Todd, Andrew and Charles we think is makes more sense to implement encryption for HDFS directly into the DistributedFileSystem client and to use CryptoFileSystem support encryption for FileSystems that don’t support native encryption.

The reasons for this change of course are:

* If we want to we add support for HDFS transparent compression, the compression should be done before the encryption (implying less entropy). If compression is to be handled by HDFS DistributedFileSystem, then the encryption has to be handled afterwards (in the write path).

* The proposed CryptoSupport abstraction significantly complicates the implementation of CryptoFileSystem and the wiring in HDFS FileSystem client.

* Building it directly into HDFS FileSystem client may allow us to avoid an extra copy of data.

Because of this, the idea is now:

* A common set of Crypto Input/Output streams. They would be used by CryptoFileSystem, HDFS encryption, MapReduce intermediate data and spills. Note we cannot use the JDK Cipher Input/Output streams directly because we need to support the additional interfaces that the Hadoop FileSystem streams implement (Seekable, PositionedReadable,  ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess,  Syncable, CanSetDropBehind).

* CryptoFileSystem.
 To support encryption in arbitrary FileSystems.

* HDFS client encryption. To support transparent HDFS encryption.

Both CryptoFilesystem and HDFS client encryption implementations would be built using the Crypto Input/Output streams, xAttributes and KeyProvider API.




> 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)