[ 
https://issues.apache.org/jira/browse/HADOOP-13578?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15657674#comment-15657674
 ] 

churro morales commented on HADOOP-13578:
-----------------------------------------

bq. I'm confused why the codec returns null for getCompressorType, 
createCompressor, getDecompressorType, and createDecompressor. This is going to 
break code that expects to wield the compressor/decompressor rather than 
streams directly. I would expect it to work more like the DefaultCodec or 
BZip2Codec where the codec returns appropriate Compressor and Decompressor 
classes and leverages the existing CompressorStream, DecompressorStream, and 
CompressionCodec.Util classes to handle the stream interfaces of Codec. For 
example, the CompressorStream abstract class already handles the single-byte 
write method, protection from double-finish and double-close, the 
DecompressorStream already implemetns skip, etc.

The reason for this was that in testing the code, namely the decompressor 
logic.  The smallest unit of work in zstandard is a block.  I noticed many 
cases during decompression where I would fill up the output buffer and still 
have some data left in my input buffer.  The data in my input buffer was not a 
complete block of compressed data and thus I was having quite a few issues when 
using the decompressor API which I felt was really clunky, you have to return 
special values to see if you need input and then you have no control over how 
much of the buffer you fill with compressed data.  For example if I have half a 
block of compressed data at the end of the input buffer, it was really 
difficult to state 

I need more data in the input buffer, because I don't have a complete 
compressed block, 
I have to move that partial compressed block to the beginning of the input 
buffer 
I also have to ensure that needsInput now returns true, even though I have 
compressed data to decompress (but not enough for a block) 

The entire streaming decompressor API is built around Bzip2 and Zlib which if 
you look have the exact same C API's and even return the same values for their 
respective inflate / deflate methods.  I would much prefer to have an 
inputstream / outputstream and just have my library compress / decompress on 
that stream itself because the zstd api does not match that of zlib or bzip2. 

bq. The original approach used direct byte buffers, but the new patch no longer 
does. All the other codecs leverage direct byte buffers, so I'm curious about 
the reasoning for that change. I'm not a JVM expert, but I'm wondering if the 
*PrimitiveArrayCritical methods have unfavorable impacts on other threads in 
the JVM (e.g.: due to pausing GC or other effects). Given that 
GetPrimitiveArrayCritical could trigger an array copy in order to perform it's 
task and we have to copy the bytes from the output buffer into the output 
stream anyway, I would expect direct byte buffers to be faster for at least the 
output buffer case.

This is from the oracle documentation:

Here is the signature of GetPrimativeArrayCritical: void * 
GetPrimitiveArrayCritical(JNIEnv *env, jarray array, jboolean *isCopy);
Note that GetPrimitiveArrayCritical might still make a copy of the array if the 
VM internally represents arrays in a different format. Therefore we need to 
check its return value against NULL for possible out of memory situations.

Looks like for most of the jvm's on linux platforms it references a memory 
address, but we can still use direct byte buffers if you feel strongly about 
it.  At least for the output buffer case as you had stated.   I didnt notice 
any performance hit on my local test environment but can look further into this.

bq. Speaking of double-finish, it looks like that could be problematic and lead 
to double-free's in the native code layer. In addition to leveraging 
CompressorStream/DecompressorStream as mentioned above to help with this, we 
could zero the stream field after we are finished and check for a non-null 
stream context before doing operations.
We should throw NPE if the caller passes a null for the source data buffer.
Similarly the native code should throw an error if it cannot obtain pointers to 
the Java buffers. Currently it just silently returns no progress which will 
result in an infinite loop in practice as it tries to reach the end of the 
input and never gets there if the error keeps occurring on each JNI invocation.

totally agree, will fix this. 

bq. The documentation for the zstd streaming interface mentions that flushing 
or ending a stream may require multiple invocations in order to fully 
accomplish the task, but I don't see corresponding loops in the java code to 
handle that scenario:

Will make sure to take care of this scenario. 

bq. IOUtils.closeQuietly is being called on the output stream which could lead 
to silent data corruption. If there was an issue writing out the last bits of 
data as a result of the close call then this will silently eat the error. The 
user is left with a "successful" operation that did not actually succeed.

Great catch, will change this. 

bq. The getLibraryName native method should do something sane for the non-UNIX 
case, like returning the Unavailable string as some codecs do when they can't 
compute it in the UNIX case. Bonus points for adding a WINDOWS case, and it 
looks like we can model after the Windows implementations of getLibraryName in 
the other codecs.

Sure we can change this.

bq. In the decompressor, why are we finished when we decode a frame? I thought 
it was valid for a ZStandard compression stream to be made up of one or more 
frames, so if there is more input after decoding a frame it seems like the 
prudent thing to do is try decoding another frame.

from the docs
*  @return : 0 when a frame is completely decoded and fully flushed, an error 
code, which can be tested using ZSTD_isError(), any other value > 0, which 
means there is still some work to do to complete the frame. The return value is 
a suggested next input size (just an hint, to help latency).

Also we can read multiple frames just fine with the decompressor.  I tested 
this by doing the following with the cli: I compressed two separate files with 
the CLI, then concatenated them together.  Then took that file and fed it into 
the ZStandardDecompressorStream.  (this concatenated file had 2 frames) and it 
decompressed just fine.  That is because finished field is only there to make 
sure we make a call to the zstd library to re-initialize the stream when we 
have reached the end of a frame.  But in the actual ZStandardDecompressorStream 
we check if there is more input to read even if we reach the end of a frame.  
The finished field is more to reinitialize the zstd stream than the actual java 
decompressor stream.

bq. It would be nice to allow the input and output buffer sizes to be 
configurable. When used as part of wide merge sorts, there are going to be a 
lot of codec instances. It may be necessary to use a smaller buffer size per 
codec to reduce the merge's memory footprint due to all those codec I/O 
buffers. It makes sense to use the library-recommended size as the default 
value, but it should be straightforward to allow the user to override this.

That shouldn't be a problem, but the zstd suggested buffer size especially for 
the decompressor is the minimum buffer size to decompress one block.  So we 
could allow the user to configure and take the larger of the 2 buffer sizes in 
case they configure it with a buffer size that is too small.

bq. Nit: srcSize is not really a size in the following code but rather a 
terminating offset. Renaming it to something like endOffset would be more 
clear, since we're comparing it against the inputBufferOffset. Same comment for 
dstSize in the decompressor.

Sure i can change that to endOffset and inputBufferOffset.  

bq. Nit: The whitespace was removed around this line in the ASF license header, 
and it would be good to restore it so it's consistent with the ASF headers in 
other files.
 * http://www.apache.org/licenses/LICENSE-2.0

Sure thing no problem. 

Thank you for such a thorough review.  I hope implementing my own streams is 
not too big of an issue.  Looking at the code, it didn't seem like it would be.

 











> Add Codec for ZStandard Compression
> -----------------------------------
>
>                 Key: HADOOP-13578
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13578
>             Project: Hadoop Common
>          Issue Type: New Feature
>            Reporter: churro morales
>            Assignee: churro morales
>         Attachments: HADOOP-13578.patch, HADOOP-13578.v1.patch, 
> HADOOP-13578.v2.patch
>
>
> ZStandard: https://github.com/facebook/zstd has been used in production for 6 
> months by facebook now.  v1.0 was recently released.  Create a codec for this 
> library.  



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to