[
https://issues.apache.org/jira/browse/HADOOP-5281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12682542#action_12682542
]
Chris Douglas commented on HADOOP-5281:
---------------------------------------
The compressors are being recycled, but GzipCodec and DefaultCodec are
improperly exchanging ZlibCompressors. In a trace printing out identityHashCode
for all compressors:
{quote}
09/03/17 01:44:46 INFO compress.CodecPool: Got brand-new compressor *12182618
GzipCodec/ZlibCompressor*
09/03/17 01:44:46 INFO compress.CodecPool: Got brand-new compressor 12376621
DefaultCodec/ZlibCompressor
09/03/17 01:44:46 INFO compress.CodecPool: Returned ZlibCompressor/12182618
09/03/17 01:44:46 INFO compress.CodecPool: Returned ZlibCompressor/12376621
09/03/17 01:44:46 INFO compress.CodecPool: Got recycled compressor 12376621
GzipCodec/ZlibCompressor
09/03/17 01:44:46 INFO compress.CodecPool: Got recycled compressor *12182618
DefaultCodec/ZlibCompressor*
{quote}
> GzipCodec fails second time it is used in a process
> ---------------------------------------------------
>
> Key: HADOOP-5281
> URL: https://issues.apache.org/jira/browse/HADOOP-5281
> Project: Hadoop Core
> Issue Type: Bug
> Affects Versions: 0.19.0
> Environment: 0.19.0 / Linux / amd64
> Reporter: Ben Maurer
> Priority: Blocker
> Attachments: 5281-0.patch, GZt.java, GZt.java
>
>
> The attached code (GZt.java) raises:
> {noformat}
> java.io.IOException: incorrect header check
> at
> org.apache.hadoop.io.compress.zlib.ZlibDecompressor.inflateBytesDirect(Native
> Method)
> at
> org.apache.hadoop.io.compress.zlib.ZlibDecompressor.decompress(ZlibDecompressor.java:221)
> at
> org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:80)
> at
> org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:74)
> at
> org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:62)
> at java.io.DataInputStream.readByte(DataInputStream.java:248)
> at org.apache.hadoop.io.WritableUtils.readVLong(WritableUtils.java:325)
> at org.apache.hadoop.io.WritableUtils.readVInt(WritableUtils.java:346)
> at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1853)
> at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1876)
> at org.apache.hadoop.io.MapFile$Reader.readIndex(MapFile.java:319)
> at org.apache.hadoop.io.MapFile$Reader.seekInternal(MapFile.java:435)
> at org.apache.hadoop.io.MapFile$Reader.seekInternal(MapFile.java:417)
> at org.apache.hadoop.io.MapFile$Reader.seek(MapFile.java:404)
> at org.apache.hadoop.io.MapFile$Reader.get(MapFile.java:523)
> {noformat}
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.