[ https://issues.apache.org/jira/browse/HADOOP-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Yi Liu updated HADOOP-10603: ---------------------------- Attachment: HADOOP-10603.3.patch Hi [~clamb] and [~tucu00], thanks for your good comments. I merge part of your attached code to the new patch, it’s good. Definitions of {{Encryptor}} and {{Decryptor}} interfaces in latest patch are as following: {code} public interface Encryptor { … … public void encrypt(ByteBuffer inBuffer, ByteBuffer outBuffer) throws IOException; } {code} For each encryption, it’s better to invoke few Cipher# methods, these calls are expensive. We could treat whole input stream or output stream as encryption/decryption unit, instead of each time encrypt/decrypt as an unit. {quote} The AESCTREncryptor/AESCTRDecryptor do update() instead of doFinal() on the cipher. This could lead to incomplete decryption of the buffer in some cipher implementations since the contract is that a doFinal() must be done. {quote} For the encryption mode which needs padding, we should use {{doFinal}}, for CTR, {{update}} is OK. The difference between these two is {{update}} will maintain the internal state, we can utilize this, so for each “encrypt/decrypt” we only need call one Cipher# method. From {{Cipher#update}} javadoc, we can know it decrypts data, for JCE default provider, and diaceros, and even UPDATE interface of OpenSSL, they all work in this way. I also agree with you that some cipher implementation may be not good, and {{update}} doesn't not encrypt/decrypt data for CTR mode, we should be able to handle this, in this situation we need to invoke {{doFinal}}. Please review the patch to see how we handle all the possible situations. In most cases, we only invoke one Cipher# method for each ecrypt/decrypt. So in the attached code {{Cipher#init}}, {{Cipher#update}}, {{Cipher#doFinal}} in each #process method is not necessary. {code} void process(Cipher cipher, int mode, SecretKeySpec key, byte[] originalIV, … cipher.init(mode, key, new IvParameterSpec(workingIV)); int mod = (int) (absolutePos % BLOCK_SIZE); cipher.update(IV_FORWARD_SINK, 0, mod, IV_FORWARD_SINK, 0); cipher.doFinal(in, out); … } {code} In the latest patch of stream implementations, For decryption: {code} @Override public int read(byte[] b, int off, int len) throws IOException { checkStream(); if ((off | len | (off + len) | (b.length - (off + len))) < 0) { throw new IndexOutOfBoundsException(); } int remaining = outBuffer.remaining(); if (remaining > 0) { int n = Math.min(len, remaining); outBuffer.get(b, off, n); return n; } else { int n = 0; if (in instanceof ByteBufferReadable) { n = ((ByteBufferReadable) in).read(inBuffer); if (n <= 0) { return n; } } else { int toRead = inBuffer.remaining(); byte[] tmp = getTmpBuf(); n = in.read(tmp, 0, toRead); if (n <= 0) { return n; } inBuffer.put(tmp, 0, n); } streamOffset += n; // Read n bytes return process(b, off, n); } } {code} We need to use {{ByteBufferReadable}}, it avoids copy. So for HDFS, it does. {quote} The current implementation assumes the counter portion always starts with zero, right? {quote} I will add this in next version of patch. For other issues, they should be resolved in latest patch, please review the patch again, thanks. We also have more test cases to cover them in HDFS-6405. I will also add test cases in common. > 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 > Reporter: Alejandro Abdelnur > Assignee: Yi Liu > Fix For: 3.0.0 > > Attachments: HADOOP-10603.1.patch, HADOOP-10603.2.patch, > HADOOP-10603.3.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)