[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007205#comment-14007205 ] Yi Liu commented on HADOOP-10603: - The new patch is [HADOOP-10603.10.patch|https://issues.apache.org/jira/secure/attachment/12646518/HADOOP-10603.10.patch] Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.10.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007457#comment-14007457 ] Charles Lamb commented on HADOOP-10603: --- Yi, I'm sorry I didn't get these comments to you sooner. In general, please add blank lines before all block comments. AESCTRCryptoCodec.java +public abstract class AESCTRCryptoCodec extends CryptoCodec { + /** + * For AES, the algorithm block is fixed size of 128 bits. + * @see http://en.wikipedia.org/wiki/Advanced_Encryption_Standard + */ + private static final int AES_BLOCK_SIZE = 16; + /** + * IV is produced by combining initial IV and the counter using addition. + * IV length should be the same as {@link #AES_BLOCK_SIZE} + */ The IV is produced by adding the initial IV to the counter. IV length should be the same as {@link #AES_BLOCK_SIZE}. + @Override + public void calculateIV(byte[] initIV, long counter, byte[] IV) { ... +ByteBuffer buf = ByteBuffer.wrap(IV); add a final decl. CryptoCodec.java + /** + * Get block size of a block cipher. Get the block size of a block cipher. + * For different algorithms, the block size may be different. + * @return int block size @return the block size + * Get a {@link #org.apache.hadoop.crypto.Encryptor}. s/Get a/Get an/ + * @return Encryptor @return the Encryptor + * @return Decryptor @return the Decryptor + * This interface is only for Counter (CTR) mode. Typically calculating + * IV(Initialization Vector) is up to Encryptor or Decryptor, for + * example {@link #javax.crypto.Cipher} will maintain encryption context + * internally when do encryption/decryption continuously using its + * Cipher#update interface. This interface is only needed for AES-CTR mode. The IV is generally calculated by the Encryptor or Decryptor and maintained as internal state. For example, a {@link #javax.crypto.Cipher} will maintain its encryption context internally using the Cipher#update interface. + * In Hadoop, multiple nodes may read splits of a file, so decrypting of + * file is not continuous, even for encrypting may be not continuous. For + * each part, we need to calculate the counter through file position. Encryption/Decryption is not always on the entire file. For example, in Hadoop, a node may only decrypt a portion of a file (i.e. a split). In these situations, the counter is derived from the file position. + * p/ + * Typically IV for a file position is produced by combining initial IV and + * the counter using any lossless operation (concatenation, addition, or XOR). + * @see http://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Counter_.28CTR.29 The IV can be calculated based on the file position by combining the initial IV and the counter with a lossless operation (concatenation, addition, or XOR). @see http://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Counter_.28CTR.29 CryptoInputStream.java +public class CryptoInputStream extends FilterInputStream implements +Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor, +CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess { Add a newline here please. + /** + * Whether underlying stream supports s/Whether underlying/Whether the underlying/ + /** + * Padding = pos%(algorithm blocksize); Padding is put into {@link #inBuffer} + * before any other data goes in. The purpose of padding is to put input data + * at proper position. s/put input data/put the input data/ + @Override + public int read(byte[] b, int off, int len) throws IOException { +int remaining = outBuffer.remaining(); final int remaining... + if (usingByteBufferRead == null) { +if (in instanceof ByteBufferReadable) { + try { +n = ((ByteBufferReadable) in).read(inBuffer); +usingByteBufferRead = Boolean.TRUE; + } catch (UnsupportedOperationException e) { +usingByteBufferRead = Boolean.FALSE; + } +} +if (!usingByteBufferRead.booleanValue()) { + n = readFromUnderlyingStream(); +} + } else { +if (usingByteBufferRead.booleanValue()) { + n = ((ByteBufferReadable) in).read(inBuffer); +} else { + n = readFromUnderlyingStream(); +} + } For the code above, I wonder if we shouldn't maintain a reference to the actual ByteBuffer once it is known to be ByteBufferReadable. If the caller switches BBs, then it is possible that this could throw a UnsupportedOperationException. So the check would be to see if the BB was the same one that was already known to be BBReadable, and if not, then check it again. + // Read data from underlying stream. + private int readFromUnderlyingStream() throws IOException { +int toRead = inBuffer.remaining(); +byte[] tmp = getTmpBuf(); +int n = in.read(tmp, 0,
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007792#comment-14007792 ] Andrew Wang commented on HADOOP-10603: -- Hey Charles, how do you feel about committing Yi's base patch, and doing these additional comments in a separate JIRA? I'm +1 on the latest rev. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.10.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007827#comment-14007827 ] Charles Lamb commented on HADOOP-10603: --- Sounds good. +1 Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.10.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007829#comment-14007829 ] Charles Lamb commented on HADOOP-10603: --- +1 Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.10.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007914#comment-14007914 ] Yi Liu commented on HADOOP-10603: - Thanks Andrew and Charles for review :-). I will commit the patch later and have a new JIRA for Charles' comment of javadoc and code comment. Let's add [~clamb] name in the contributor list when commit since he makes many contributions to this JIRA too. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.10.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007960#comment-14007960 ] Yi Liu commented on HADOOP-10603: - I have just committed this to branch, Thanks all. I filed HADOOP-10628 to address [~clamb]'s comment of javadoc and few code style. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.10.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14005952#comment-14005952 ] Yi Liu commented on HADOOP-10603: - Charles, thanks for your good comments: {quote} I have also made some edits to CryptoInputStream and CryptoOutputStream. I have attached the whole file for those two rather than diffs. {quote} Thanks for revise javadoc/comments, I have merged them to the new patch. You add {{getWrappedStream}}, I also put into the new patch, it’s for test case? {quote} CryptoFactory.java Perhaps rename this to Crypto. {quote} I change it to {{CryptoCodec}}. {quote} getEncryptor/getDecryptor should also declare throws GeneralSecurityException {quote} OK, I throw it out now, originally I catch it internally. {quote} Encryptor.java encrypt should declare throws GeneralSecurityException {quote} I already wrap it to {{IOException}} {quote} decl for encrypt 80 chars {quote} OK. I will update it. {quote} Consider making this interface an inner class of Crypto (aka CryptoFactory). {quote} The {{Encryptor}}/{{Decryptor}} contains more than one interfaces, they are not suitable as inner class. {quote} Remind me again why encrypt/decrypt don't take a position argument? {quote} Several reasons: * We don’t need do {{Cipher#init}} + {{Cipher#update}} + {{Cipher#doFinal}} for every {{encrypt/decrypt}} operation, that’s expensive. We should reply on the {{Cipher}} maintaining the encryption/decryption context, such as calculating counter/IV, we just need {{Cipher#update}} for CTR, only for bad JCE provider implementation, we need {{Cipher#doFinal}}, and we can handle this situation. And I believe it will never happen, it should be a bug of the cipher provider since it doesn’t follow the definition of {{Cipher#update}}. So we don’t need have a position argument. * The interface is a common interface, we should think other encryption mode may be used by other features in future, a position argument doesn’t make sense to other mode. {quote} I wonder if, in general, we'll also want byte[] overloadings of the methods (as well as BB) for encrypt()/decrypt(). {quote} We can have this, if you prefer or some other guy prefer, let’s add it. {quote} The decl for decrypt 80 chars {quote} Right, I will updated it. {quote} JCEAESCTRCryptoFactory.java This file needs an apache license header Perhaps rename it to JCEAESCTRCrypto.java getDescryptor/getEncryptor should throw GeneralSecurityException {quote} Right, I will update it. And rename to {{JCEAESCTRCryptoCodec}} {quote} JCEAESCTRDecryptor.java ctor should throw GeneralSecurityException instead of RTException decrypt should throw GeneralSecurityException JCEAESCTREncryptor.java ctor should throw GeneralSecurityException instead of RTException encrypt should throw GeneralSecurityException {quote} I will update constructor to throw {{GeneralSecurityException}}, but for {{decrypt/encrypt}} I have wrapped it to {{IOException}}. {quote} put a newline after public class CryptoUtils { Could calIV be renamed to calcIV? {quote} calIV has been refined and renamed to {{calculateIV}}, for different CryptoCodec, we can have different implementation. {quote} CryptoFSDataOutputStream.java Why is fsOut needed? Why can't you just reference out for (e.g.) getPos()? {quote} Since {{out}} is instanceOf {{CryptoOutputStream}}, doesn’t have {{getPos()}} {quote} CryptoInputStream.java You'll need a getWrappedStream() method. {quote} Yes, I add it, but I’m not quite clear the purpose? {quote} Why 8192? Should this be moved to a static final int CONSTANT? {quote} It’s a configuration now. {quote} IWBNI the name of the interface that a particular method is implementing were put in a comment before the @Override. For instance, // PositionedRead @Override public int read(long position ...) {quote} OK, I will update it. {quote} In read(byte[], int, int), isn't the if (!usingByteBufferRead) I am worried that throwing and catching UnsupportedOperationException will be expensive. It seems very likely that for any particular stream, the same byte buffer will be passed in for the life of the stream. That means that for every call to read(...) there is potential for the UnsupportedOperationException to be thrown. That will be expensive. Perhaps keep a piece of state in the stream that gets set on the first time through indicating whether the BB is readable or not. Or keep a reference to the BB along with a bool. If the reference changes (on the off chance that the caller switched BBs for the same stream), then you can redetermine whether read is supported or not. {quote} Actually we have {{in instanceof ByteBufferReadable}}, not for any stream, so it’s not expensive. if a stream implements {{ByteBufferReadable}}, why we need {{UnsupportedOperationExation}}? Since it could also throw {{UnsupportedOperationException}} too, it may be a wrapper
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14006624#comment-14006624 ] Charles Lamb commented on HADOOP-10603: --- Hi Yi, Good work so far. I took your latest patch and incorporated it into my sandbox and got my unit tests running with it. I have also made some edits to CryptoInputStream and CryptoOutputStream. I have attached the whole file for those two rather than diffs. CryptoFactory.java Perhaps rename this to Crypto. getEncryptor/getDecryptor should also declare throws GeneralSecurityException Encryptor.java encrypt should declare throws GeneralSecurityException decl for encrypt 80 chars Consider making this interface an inner class of Crypto (aka CryptoFactory). Remind me again why encrypt/decrypt don't take a position argument? I wonder if, in general, we'll also want byte[] overloadings of the methods (as well as BB) for encrypt()/decrypt(). Decryptor.java decrypt should throw GeneralSecurityException The decl for decrypt 80 chars Consider making this interface a subclass of Crypto (aka CryptoFactory). JCEAESCTRCryptoFactory.java This file needs an apache license header Perhaps rename it to JCEAESCTRCrypto.java getDescryptor/getEncryptor should throw GeneralSecurityException JCEAESCTRDecryptor.java ctor should throw GeneralSecurityException instead of RTException decrypt should throw GeneralSecurityException JCEAESCTREncryptor.java ctor should throw GeneralSecurityException instead of RTException encrypt should throw GeneralSecurityException CryptoUtils.java put a newline after public class CryptoUtils { Could calIV be renamed to calcIV? CryptoFSDataOutputStream.java Why is fsOut needed? Why can't you just reference out for (e.g.) getPos()? CryptoInputStream.java You'll need a getWrappedStream() method. Why 8192? Should this be moved to a static final int CONSTANT? IWBNI the name of the interface that a particular method is implementing were put in a comment before the @Override. For instance, // PositionedRead @Override public int read(long position ...) IWBNI all of the methods for a particular interface were grouped together in the code. In read(byte[], int, int), isn't the if (!usingByteBufferRead) I am worried that throwing and catching UnsupportedOperationException will be expensive. It seems very likely that for any particular stream, the same byte buffer will be passed in for the life of the stream. That means that for every call to read(...) there is potential for the UnsupportedOperationException to be thrown. That will be expensive. Perhaps keep a piece of state in the stream that gets set on the first time through indicating whether the BB is readable or not. Or keep a reference to the BB along with a bool. If the reference changes (on the off chance that the caller switched BBs for the same stream), then you can redetermine whether read is supported or not. In readFully, you could simplify the implementation by just calling into read(long, byte[]...), like this: @Override // PositionedReadable public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { int nread = 0; while (nread length) { int nbytes = read(position + nread, buffer, offset + nread, length - nread); if (nbytes 0) { throw new EOFException(End of file reached before reading fully.); } nread += nbytes; } } That way you can let read(long...) do all the unwinding of the seek position. In seek(), you can do a check for forward == 0 and return immediately, thus saving the two calls to position() in the noop case. Ditto skip(). I noticed that you implemented read(ByteBufferPool), but not releaseBuffer(BB). Is that because you didn't have time (it's ok if that's the case, I'm just wondering why one and not the other)? CryptoOutputStream.java You'll need a getWrappedStream() method. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch A common set of Crypto Input/Output streams. They would be used by CryptoFileSystem, HDFS encryption, MapReduce intermediate
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14006628#comment-14006628 ] Charles Lamb commented on HADOOP-10603: --- CryptoInputStream.java: Shouldn't usingByteBufferRead be a class variable so that we don't keep checking in instanceof ByteBufferReadable everytime we call read()? Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14006655#comment-14006655 ] Andrew Wang commented on HADOOP-10603: -- Thanks for the mega-rev Yi, I went through and ticked off my previous review comments. I think we're pretty close if Charles agrees, just had a few things besides the last few you already identified. - New configuration keys should go in CommonConfigurationKeysPublic, with a provided default also. - Any reason you put the buffer size in CryptoCodec rather than in the Crypto streams? The streams seem to make more sense. - Could also do some basic Precondition validation on the config parameters. - Should CryptoCodec do {{setConf(new Configuration())}} in its constructor? - Streams still have some hardcoded {{16}} - (off+len) can still int overflow, need to do some casting to longs to be safe, or some tricks to avoid addition - updateDecryptor still doesn't need that parameter - Still some tabs present (I think your IDE inserts them when splitting a string) Test: * getDataLen() is never used * Let's add conservative test timeouts (e.g. 12) * I think you can use the @Ignore annotation to skip unsupported LocalFS tests. Can provide a reason too. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14006678#comment-14006678 ] Yi Liu commented on HADOOP-10603: - Thanks Charles. {quote} Shouldn't usingByteBufferRead be a class variable so that we don't keep checking in instanceof ByteBufferReadable everytime we call read()? {quote} Right, I will update this. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14006680#comment-14006680 ] Yi Liu commented on HADOOP-10603: - Thanks [~andrew.wang] for your nice comments. I will update the patch for your new comments together with the left few items, and will respond you later. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.9.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14004498#comment-14004498 ] Yi Liu commented on HADOOP-10603: - Andrew, thanks for your detailed review. Although they are based on a slightly old version, but most are also valid for latest patch :-). HADOOP-10617 is the common side test cases for crypto streams, since we already have lots of test cases and still need to increase and it's a bit large if merging to this, I made it as separate JIRA. Sure, according to your suggestion, I will merge it to this JIRA. Following is my response to your comments, and I will update the patch later. {quote} Need class javadoc and interface annotations on all new classes Need p/ to actually line break in javadoc Some tab characters present {quote} I will update them. {quote} s/mod/mode What does calIV mean? Javadoc here would be nice. calIV would be simpler if we used ByteBuffer.wrap and getLong. I think right now, we also need to cast each value to a long before shifting, else it only works up to an int. Would be good to unit test this function. {quote} Right, I will update them. {quote} Could you define the term block in the #encrypt javadoc? {quote} It was a wrong word and should be “buffer”. Already updated it in latest patch. {quote} I don't understand the reinit conditions, do you mind explaining this a bit? The javadoc for Cipher#update indicates that it always fully reads the input buffer, so is the issue that the cipher sometimes doesn't flush all the input to the output buffer? {quote} Andrew, I agree with you. The javadoc for Cipher#update indicates that it always fully reads the input buffer and decrypt all input data. This will be always correct for CTR mode, for some of other modes input data may be buffered if requested padding (CTR doesn’t need padding). Charles has concern about maybe some custom JCE provider implementation can’t decrypt all data for CTR mode using {{Cipher#update}}, so I add the reinit conditions, and I think if that specific provider can’t decrypt all input data of {{Cipher#update}} for CTR mode, that should be a bug of that provider since it doesn't follow the definition of {{Cipher#update}}. {quote} If this API only accepts direct ByteBuffers, we should Precondition check that in the implementation {quote} I’m not sure we have this restriction. Java heap byteBuffer is also OK. Direct ByteBuffer is more efficient (no copy) when the cipher provider is native code and using JNI. I will add if you prefer. {quote} Javadoc for {{encrypt}} should link to {{javax.crypto.ShortBufferException}}, not {{#ShortBufferException}}. I also don't see this being thrown because we wrap everything in an IOException. {quote} Right, I will revise this. {quote} How was the default buffer size of 8KB chosen? This should probably be a new configuration parameter, or respect io.file.buffer.size. {quote} OK. I will add configuration parameter for the default buffer size. {quote} Potential for int overflow in {{#write}} where we check {{off+len 0}}. I also find this if statement hard to parse, would prefer if it were expanded. {quote} OK. I will expand them in next patch. {quote} Is the {{16}} in {{updateEncryptor}} something that should be hard-coded? Maybe pull it out into a constant and javadoc why it's 16. I'm curious if this is dependent on the Encryptor implementation. {quote} Let’s pull it out into variable. 16bytes is 128bits, and it’s in definition of AES: http://en.wikipedia.org/wiki/Advanced_Encryption_Standard. Let’s define it as a configuration parameter, since other algorithm may have different block size, although we use AES. {quote} We need to be careful with direct BBs, since they don't trigger GC. We should be freeing them manually when the stream is closed, or pooling them somehow for reuse. {quote} Good point. For pooling them, maybe they are created with different buffer size and not suitable in pool? So I will add freeing them manually when the stream is closed. {quote} • In {{#process}}, we flip the inBuf, then if there's no data we just return. Shouldn't we restore inBuf to its previous padded state first? Also, IIUC {{inBuffer.remaining()}} cannot be less than padding since the inBuffer position does not move backwards, so I'd prefer to see a Precondition check and {{inBuf.remaining() == padding)}}. Test case would be nice if I'm right about this. {quote} You are right, there is a potential issue. I will fix it and add test case. Since in our code, only when we have input data then we go to {{#process}}, so {{inBuffer}} should have real data. But from view of code logic we should handle like you said. And agree we have a precondition check. {quote} Rename {{#process}} to {{#encrypt}}? {quote} Good, let’s do that. {quote} Do we need the special-case logic with tmpBuf? It looks like
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14005400#comment-14005400 ] Charles Lamb commented on HADOOP-10603: --- Hi Yi, Good work so far. I took your latest patch and incorporated it into my sandbox and got my unit tests running with it. I have also made some edits to CryptoInputStream and CryptoOutputStream. I have attached the whole file for those two rather than diffs. CryptoFactory.java Perhaps rename this to Crypto. getEncryptor/getDecryptor should also declare throws GeneralSecurityException Encryptor.java encrypt should declare throws GeneralSecurityException decl for encrypt 80 chars Consider making this interface an inner class of Crypto (aka CryptoFactory). Remind me again why encrypt/decrypt don't take a position argument? I wonder if, in general, we'll also want byte[] overloadings of the methods (as well as BB) for encrypt()/decrypt(). Decryptor.java decrypt should throw GeneralSecurityException The decl for decrypt 80 chars Consider making this interface a subclass of Crypto (aka CryptoFactory). JCEAESCTRCryptoFactory.java This file needs an apache license header Perhaps rename it to JCEAESCTRCrypto.java getDescryptor/getEncryptor should throw GeneralSecurityException JCEAESCTRDecryptor.java ctor should throw GeneralSecurityException instead of RTException decrypt should throw GeneralSecurityException JCEAESCTREncryptor.java ctor should throw GeneralSecurityException instead of RTException encrypt should throw GeneralSecurityException CryptoUtils.java put a newline after public class CryptoUtils { Could calIV be renamed to calcIV? CryptoFSDataOutputStream.java Why is fsOut needed? Why can't you just reference out for (e.g.) getPos()? CryptoInputStream.java You'll need a getWrappedStream() method. Why 8192? Should this be moved to a static final int CONSTANT? IWBNI the name of the interface that a particular method is implementing were put in a comment before the @Override. For instance, // PositionedRead @Override public int read(long position ...) IWBNI all of the methods for a particular interface were grouped together in the code. In read(byte[], int, int), isn't the if (!usingByteBufferRead) I am worried that throwing and catching UnsupportedOperationException will be expensive. It seems very likely that for any particular stream, the same byte buffer will be passed in for the life of the stream. That means that for every call to read(...) there is potential for the UnsupportedOperationException to be thrown. That will be expensive. Perhaps keep a piece of state in the stream that gets set on the first time through indicating whether the BB is readable or not. Or keep a reference to the BB along with a bool. If the reference changes (on the off chance that the caller switched BBs for the same stream), then you can redetermine whether read is supported or not. In readFully, you could simplify the implementation by just calling into read(long, byte[]...), like this: @Override // PositionedReadable public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { int nread = 0; while (nread length) { int nbytes = read(position + nread, buffer, offset + nread, length - nread); if (nbytes 0) { throw new EOFException(End of file reached before reading fully.); } nread += nbytes; } } That way you can let read(long...) do all the unwinding of the seek position. In seek(), you can do a check for forward == 0 and return immediately, thus saving the two calls to position() in the noop case. Ditto skip(). I noticed that you implemented read(ByteBufferPool), but not releaseBuffer(BB). Is that because you didn't have time (it's ok if that's the case, I'm just wondering why one and not the other)? CryptoOutputStream.java You'll need a getWrappedStream() method. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.patch 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
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14005481#comment-14005481 ] Yi Liu commented on HADOOP-10603: - Thanks Charles for good comments. I'm refining the patch for Andrew's comments, will respond you later and also want to address your comments in the new patch :-) Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: CryptoInputStream.java, CryptoOutputStream.java, HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.5.patch, HADOOP-10603.6.patch, HADOOP-10603.7.patch, HADOOP-10603.8.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14004190#comment-14004190 ] Andrew Wang commented on HADOOP-10603: -- Hi Yi, thanks for working on this code. I have some review comments based on a slightly old version of this patch. Throughout, if I've correctly identified a correctness issue, let's have a corresponding unit test if possible. I'd personally also rather not split the implementation and tests into separate JIRAs. Nitty stuff: * Need class javadoc and interface annotations on all new classes * Need p/ to actually line break in javadoc * Some tab characters present CryptoUtils * s/mod/mode * What does calIV mean? Javadoc here would be nice. * calIV would be simpler if we used ByteBuffer.wrap and {{getLong}}. I think right now, we also need to cast each value to a {{long}} before shifting, else it only works up to an int. Would be good to unit test this function. Encryptor/JCEAESCTREncryptor: * Could you define the term block in the {{#encrypt}} javadoc? * I don't understand the reinit conditions, do you mind explaining this a bit? The javadoc for {{Cipher#update}} indicates that it always fully reads the input buffer, so is the issue that the cipher sometimes doesn't flush all the input to the output buffer? * If this API only accepts direct ByteBuffers, we should Precondition check that in the implementation * Javadoc for {{encrypt}} should link to {{javax.crypto.ShortBufferException}}, not {{#ShortBufferException}}. I also don't see this being thrown because we wrap everything in an IOException. CryptoOutputStream: * How was the default buffer size of 8KB chosen? This should probably be a new configuration parameter, or respect io.file.buffer.size. * Potential for int overflow in {{#write}} where we check {{off+len 0}}. I also find this if statement hard to parse, would prefer if it were expanded. * Is the {{16}} in updateEncryptor something that should be hard-coded? Maybe pull it out into a constant and javadoc why it's 16. I'm curious if this is dependent on the Encryptor implementation. * We need to be careful with direct BBs, since they don't trigger GC. We should be freeing them manually when the stream is closed, or pooling them somehow for reuse. * In {{#process}}, we flip the inBuf, then if there's no data we just return. Shouldn't we restore inBuf to its previous padded state first? Also, IIUC {{inBuffer.remaining()}} cannot be less than padding since the inBuffer position does not move backwards, so I'd prefer to see a Precondition check and {{inBuf.remaining() == padding)}}. Test case would be nice if I'm right about this. * Rename {{#process}} to {{#encrypt}}? * Do we need the special-case logic with tmpBuf? It looks like outBuffer is always direct. * Do we need to update padding when we do a flush? * Also in {{#flush}}, s/encryption/encrypt * oneByte can be final * If you have extra time, an ASCII art diagram showing how {{padding}} and the stream offset works would also be nice. Javadoc for the special padding handling would be nice. * Can make class-private methods private * Should {{close()}} also close the underlying stream? CryptoInputStream: * Many of the OutputStream comments apply here too: hardcoded 8KB buffer size, int overflow for {{off+len}}, hardcoded 16, handling of padding in {{process}}, freeing direct buffers, private methods, closing underlying stream, etc. * Do we have tests for wrapping both ByteBufferReadable and not streams? * Rename {{process()}} to {{decrypt()}}? * In {{process}}, the catch/throw there doesn't seem useful, since everything that throws already throws an IOException. * Positioned read and readFully, doing a seek in the finally will not work on a non-Seekable stream. There's also no need to catch {{ClassCastException}} since it's already handled in {{seek}}. * {{readFully(long, byte[])}} should just delegate directly to the other readFully method, it doesn't need to do anything else. * updateDecryptor doesn't seem to need the {{long offset}} parameter since it's always passed {{streamOffset}}. * We need to return -1 on EOF for zero-byte reads, see HDFS-5762. * Comment in {{skip}} about why we subtract then add {{outBuffer.remaining()}} would be good. * Some empty {{throw new UnsupportedOperationException()}} could use text Decryptor: * s/if initialize fails/if initialization fails/ Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14001126#comment-14001126 ] Yi Liu commented on HADOOP-10603: - The tests for crypto input and output streams are in HADOOP-10617 and HDFS-6405. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Affects Versions: fs-encryption (HADOOP-10150 and HDFS-6134) Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: fs-encryption (HADOOP-10150 and HDFS-6134) Attachments: HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.3.patch, HADOOP-10603.4.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13998767#comment-13998767 ] Yi Liu commented on HADOOP-10603: - In HDFS-6392, Wire crypto streams for encrypted files in DFSClient. So {{Seekable}}, {{PositionedReadable}} ... should be implemented in {{CryptoInputStream}} instead of internally in {{CryptoFSDataInputStream}}. Furthermore, refining to have similar interfaces like {{DirectDecompressor}} for Encryptor/Decryptor almost finish, will update the patch later.. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: 3.0.0 Attachments: HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13999474#comment-13999474 ] Yi Liu commented on HADOOP-10603: - Thanks [~clamb] and [~tucu00] , good comments. Actually I have also refined crypto streams using similar interfaces like {{DirectDecompressor}}, I will merge your proposed code to mine, and update later today. I will response you later today, too. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: 3.0.0 Attachments: HADOOP-10603.1.patch, HADOOP-10603.2.patch, HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13998295#comment-13998295 ] Yi Liu commented on HADOOP-10603: - This patch is split from HADOOP-10150 with a bit modification, it’s *not* a final patch and I’m improving it. The {{Encryptor}} and {{Decryptor}} interfaces in the patch are similar with {{Compressor}} and {{Decompressor}} interfaces. There is also a {{DirectDecompressor}} interface having: {code} public void decompress(ByteBuffer src, ByteBuffer dst) throws IOException; {code} This can avoid some bytes copy, should we define {{Encryptor}} and {{Decryptor}} in this way instead of definition in the patch? {{CryptoFSDataOutputStream}} extends and wrap {{FSDataOutputStream}}, and {{CryptoFSDataInputStream}} extends and wrap {{FSDataInputStream}}. They can be used in Hadoop FileSystem to supply encryption/decryption functionalities. The test cases will be updated after we finalize. I have another JIRA to cover testing crypto streams in HDFS. Thoughts? Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: 3.0.0 Attachments: HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (HADOOP-10603) Crypto input and output streams implementing Hadoop stream interfaces
[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13998428#comment-13998428 ] Yi Liu commented on HADOOP-10603: - In HADOOP-10047, @Gopal V add a directbuffer Decompressor API to hadoop: {quote} With the Zero-Copy reads in HDFS (HDFS-5260), it becomes important to perform all I/O operations without copying data into byte[] buffers or other buffers which wrap over them. This is a proposal for adding a DirectDecompressor interface to the io.compress, to indicate codecs which want to surface the direct buffer layer upwards. The implementation should work with direct heap/mmap buffers and cannot assume .array() availability. {quote} So we can have similar API for encryption/decryption. In HADOOP-10591, direct buffer allocating/deallocating is discussed. We should have same consideration. BTW, crypto streams test in HDFS is addressed in HDFS-6405. We will also have common test for this JIRA. Crypto input and output streams implementing Hadoop stream interfaces - Key: HADOOP-10603 URL: https://issues.apache.org/jira/browse/HADOOP-10603 Project: Hadoop Common Issue Type: Sub-task Components: security Reporter: Alejandro Abdelnur Assignee: Yi Liu Fix For: 3.0.0 Attachments: HADOOP-10603.patch 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). -- This message was sent by Atlassian JIRA (v6.2#6252)