[
https://issues.apache.org/jira/browse/HADOOP-18799?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Frens Jan Rumph updated HADOOP-18799:
-------------------------------------
Summary: zstd compressor fails with Src size is incorrect (was: zstd
compressor fails with)
> zstd compressor fails with Src size is incorrect
> ------------------------------------------------
>
> Key: HADOOP-18799
> URL: https://issues.apache.org/jira/browse/HADOOP-18799
> Project: Hadoop Common
> Issue Type: Bug
> Components: native
> Affects Versions: 3.3.0
> Reporter: Frens Jan Rumph
> Priority: Major
>
> It seems like I've hit an issue similar to
> https://issues.apache.org/jira/browse/HADOOP-15822. I haven't been able to
> reproduce the issue though. I did manage to add a little bit of logging to
> org.apache.hadoop.io.compress.zstd.ZStandardCompressor. I've captured the off
> and len arguments of compress and the srcOffset and srcLen arguments for
> deflateBytesDirect:
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 0 131591}}
> {{compress 0 131591}}
> {{deflateBytesDirect 131072 519}}
> Just after that last line the process dies with a java.lang.InternalError:
> Src size is incorrect:
> {{org.apache.hadoop.mapred.YarnChild: Error running child :
> java.lang.InternalError: Src size is incorrect}}
> {{at
> org.apache.hadoop.io.compress.zstd.ZStandardCompressor.deflateBytesDirect(Native
> Method)}}
> {{at
> org.apache.hadoop.io.compress.zstd.ZStandardCompressor.compress(ZStandardCompressor.java:220)}}
> {{at
> org.apache.hadoop.io.compress.CompressorStream.compress(CompressorStream.java:81)}}
> {{at
> org.apache.hadoop.io.compress.CompressorStream.write(CompressorStream.java:76)}}
> {{at
> java.base/java.io.BufferedOutputStream.write(BufferedOutputStream.java:123)}}
> {{at java.base/java.io.DataOutputStream.write(DataOutputStream.java:107)}}
> {{at
> org.apache.hadoop.io.SequenceFile$BlockCompressWriter.writeBuffer(SequenceFile.java:1569)}}
> {{...}}
> I have also seen this error: java.lang.InternalError: Error (generic):
> {{java.lang.InternalError: Error (generic)}}
> {{at
> org.apache.hadoop.io.compress.zstd.ZStandardCompressor.deflateBytesDirect(Native
> Method)}}
> {{at
> org.apache.hadoop.io.compress.zstd.ZStandardCompressor.compress(ZStandardCompressor.java:220)}}
> {{at
> org.apache.hadoop.io.compress.CompressorStream.compress(CompressorStream.java:81)}}
> {{at
> org.apache.hadoop.io.compress.CompressorStream.write(CompressorStream.java:76)}}
> {{at
> java.base/java.io.BufferedOutputStream.write(BufferedOutputStream.java:123)}}
> {{at java.base/java.io.DataOutputStream.write(DataOutputStream.java:107)}}
> {{at
> org.apache.hadoop.io.SequenceFile$BlockCompressWriter.writeBuffer(SequenceFile.java:15}}
> {{...}}
> Note that the arguments `131072 519` are _always_ given to
> `deflateBytesDirect` in case things go wrong. In other cases the offset
> argument is zero and the size argument is smaller, but not zero; e.g., 0 and
> 7772.
> As for some context: we're using the compression as part of writing sequence
> files with data serialised with Kryo to Backblaze using the S3A file system /
> S3 client with a map-reduce job on YARN. The job has no issues with smaller
> values, but for larger ones this situation happens. I've seen very larges
> values being written successfully, but at some point this error is raised all
> over the place (after a few larger values). Perhaps some buffer is filling up?
> Unfortunately, I'm developing using a Mac with M1 processor. So reproducing
> the issue locally is not a simple feat. If I can somehow produce more leads
> to investigate this, I'd be happy to.
> As an aside: we're considering working around this using the
> hbase-compression-zstd module. This is an alternative compression codec that
> uses the zstd-jni library without depending on hadoop native.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]