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