[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 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)