[ 
https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 outBuffer is 
always direct.
{quote}
Right, let’s remove the unnecessary logic.

{quote}
Do we need to update padding when we do a flush?
{quote}
We don’t need to update padding when doing {{flush}}.  In {{flush}}, we will 
encrypt the remaining data, we don’t need to recalculate the padding.

{quote}
 Also in {{#flush}}, s/encryption/encrypt 
{{oneByte}} can be final
{quote}
Right, I will update it.

{quote}
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.
{quote}
Good point, I will add a diagram.

{quote}
Can make class-private methods private
{quote}
OK. I will update that.

{quote}
Should {{close()}} also close the underlying stream?
{quote}
Actually in {{#close}} we have called {{super.close}}, it will close the 
underlying stream.

{quote}
Do we have tests for wrapping both ByteBufferReadable and not streams?
{quote}
Please refer to the latest patch, we refine {{ByteBufferReadable}} and support 
{code}public boolean seekToNewSource(long targetPos) throws IOException{code}
{code}public ByteBuffer read(ByteBufferPool bufferPool, int maxLength, 
EnumSet<ReadOption> opts) throws IOException, 
UnsupportedOperationException{code}
{code}public void releaseBuffer(ByteBuffer buffer){code}
And we also have test cases to cover them.

{quote}
Rename {{process()}} to {{decrypt()}}?
{quote}
Right, I will do that.

{quote}
In process, the catch/throw there doesn't seem useful, since everything that 
throws already throws an IOException.
{quote}
I will refine this.

{quote}
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}}.
{quote}
OK, I will handle it using Positioned read and readFully of underlying stream. 
The position of using {{seek}} mades logic more simple, since will should not 
change the underlying stream position when using Positioned read and readFully. 
Furthermore, I saw in {{FSInputStream}} has similar logic.
Anyway, I will refine this not using {{seek}}.

{quote}
{{readFully(long, byte[])}} should just delegate directly to the other 
readFully method, it doesn't need to do anything else.
{quote}
You are right, the exception have been checked in delegated method.

{quote}
{{updateDecryptor}} doesn't seem to need the {{long offset}} parameter since 
it's always passed {{streamOffset}}.
{quote}
Right, I will update it.

{quote}
We need to return -1 on EOF for zero-byte reads, see .
{quote}
Great, I will update it.

{quote}
Comment in skip about why we subtract then add outBuffer.remaining() would be 
good.
{quote}
OK, I will add it.

{quote}
Some empty throw new UnsupportedOperationException() could use text
{quote}
OK

{quote}
s/if initialize fails/if initialization fails/
{quote}
OK

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

Reply via email to