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