[
https://issues.apache.org/jira/browse/HADOOP-12990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17032199#comment-17032199
]
John Zhuge edited comment on HADOOP-12990 at 2/8/20 6:31 AM:
-------------------------------------------------------------
OOM usually indicates format mismatch, e.g., reading a large block size, then
trying to allocate memory.
After looking into Spark code, I realized I was wrong about using Hadoop codec.
Spark uses its own LZ4 codec based on
[lz4-java|https://github.com/lz4/lz4-java]. Check out [LZ4CompressionCodec in
2.3.4|https://github.com/apache/spark/blob/v2.3.4/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala#L113-L124].
Its javadoc points out:
{quote} * @note The wire protocol for this codec is not guaranteed to be
compatible across versions
* of Spark. This is intended for use as an internal compression utility within
a single Spark
* application.{quote}
Not sure whether lz4-java LZ4BlockOutputStream output can be read by Linux lz4
tool.
Your best bet may be writing a Java decompression application with a compatible
version of lz4-java, e.g., 1.4.0 used by Spark 2.3.
was (Author: jzhuge):
OOM usually indicates format mismatch, e.g., reading a large block size, then
trying to allocate memory.
After looking into Spark code, I realized I was wrong about using Hadoop codec.
Spark uses its own LZ4 codec based on
[lz4-java|https://github.com/lz4/lz4-java]. Check out [LZ4CompressionCodec in
2.3.4|https://github.com/apache/spark/blob/v2.3.4/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala#L113-L124].
Its javadoc points out:
{quote} * @note The wire protocol for this codec is not guaranteed to be
compatible across versions
* of Spark. This is intended for use as an internal compression utility within
a single Spark
* application.{quote}
Not sure whether lz4-java LZ4BlockOutputStream output can be read by Linux lz4
tool.
Your best bet may be writing a Java decompression application with a compatible
version of lz4-java, e.g., Spark 2.3 uses lz4-java 1.4.0.
> lz4 incompatibility between OS and Hadoop
> -----------------------------------------
>
> Key: HADOOP-12990
> URL: https://issues.apache.org/jira/browse/HADOOP-12990
> Project: Hadoop Common
> Issue Type: Bug
> Components: io, native
> Affects Versions: 2.6.0
> Reporter: John Zhuge
> Priority: Minor
>
> {{hdfs dfs -text}} hit exception when trying to view the compression file
> created by Linux lz4 tool.
> The Hadoop version has HADOOP-11184 "update lz4 to r123", thus it is using
> LZ4 library in release r123.
> Linux lz4 version:
> {code}
> $ /tmp/lz4 -h 2>&1 | head -1
> *** LZ4 Compression CLI 64-bits r123, by Yann Collet (Apr 1 2016) ***
> {code}
> Test steps:
> {code}
> $ cat 10rows.txt
> 001|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 002|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 003|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 004|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 005|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 006|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 007|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 008|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 009|c1|c2|c3|c4|c5|c6|c7|c8|c9
> 010|c1|c2|c3|c4|c5|c6|c7|c8|c9
> $ /tmp/lz4 10rows.txt 10rows.txt.r123.lz4
> Compressed 310 bytes into 105 bytes ==> 33.87%
> $ hdfs dfs -put 10rows.txt.r123.lz4 /tmp
> $ hdfs dfs -text /tmp/10rows.txt.r123.lz4
> 16/04/01 08:19:07 INFO compress.CodecPool: Got brand-new decompressor [.lz4]
> Exception in thread "main" java.lang.OutOfMemoryError: Java heap space
> at
> org.apache.hadoop.io.compress.BlockDecompressorStream.getCompressedData(BlockDecompressorStream.java:123)
> at
> org.apache.hadoop.io.compress.BlockDecompressorStream.decompress(BlockDecompressorStream.java:98)
> at
> org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:85)
> at java.io.InputStream.read(InputStream.java:101)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:85)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:59)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:119)
> at org.apache.hadoop.fs.shell.Display$Cat.printToStdout(Display.java:106)
> at org.apache.hadoop.fs.shell.Display$Cat.processPath(Display.java:101)
> at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:317)
> at
> org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:289)
> at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:271)
> at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:255)
> at
> org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:118)
> at org.apache.hadoop.fs.shell.Command.run(Command.java:165)
> at org.apache.hadoop.fs.FsShell.run(FsShell.java:315)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84)
> at org.apache.hadoop.fs.FsShell.main(FsShell.java:372)
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]