[
https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 stream such as
{{FSDataInputStream}}, whether a {{FSDataInputStream}} supports {{ByteBuffer}}
read, depends on the it’s wrapped stream.
{quote}
In readFully, you could simplify the implementation by just calling into
read(long, byte[]...), like this:
{quote}
I have refined {{readFully}} to not rely on {{seek}}. For original
implementation, if {{readFully(long position..}} utilize {{read(long
position…}}, it is not good, since the later has worse performance than
{{read(byte[]..}}.
{quote}
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().
{quote}
Right, I will check for 0 separately to make it more efficient.
{quote}
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)?
{quote}
I have implemented {{releaseBuffer}}, we just need to do
{{((HasEnhancedByteBufferAccess) in).releaseBuffer(buffer);}}, it was already
in the patch you reviewed.
> 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)