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

Andrew Kyle Purtell edited comment on HBASE-27706 at 3/16/23 10:36 PM:
-----------------------------------------------------------------------

HBase native codecs all work by appending to a buffer and then compressing the 
whole buffer in one shot when the BlockCompressionStream calls the codec 
finish() method. This is how lz4 and snappy and some other codecs work both in 
Hadoop native and HBase implementations, and it was extended to all cases for 
simplicity. We use the Zstandard one shot API.
||BlockCompressionStream calls this||Codec does this||
|compress|buffer|
|compress|buffer|
|compress|buffer|
|finish|ZSTD_compress2 (via zstd-jni's Zstd#compress())|

One of the reasons we do that is aircompressor's limited zstandard support only 
operates in one-shot mode and does not offer a streaming API compatible with 
the native C zstandard library.

The hadoop native zstd codec uses the zstandard C library's streaming API, 
ZSTD_compressStream and ZSTD_decompressStream, basically wrapping the hierarchy 
of compression related Java stream APIs around this underlying native stream 
API.
||BlockCompressionStream calls this||Codec does this||
|compress|ZSTD_compressStream|
|compress|ZSTD_compressStream|
|compress|ZSTD_compressStream|
|finish|ZSTD_finishStream|

So what you get framed in the blocks of BlockCompressionStream is different 
enough to not be compatible.

In theory a new HBase native zstandard codec could do the same sequence of 
operations as Hadoop's native one by using zstd-jni's 
ZstdDirectBufferCompressingStream and ZstdDirectBufferDecompressingStream 
instead. I am not sure the behavior would be 100% identical but maybe 
compatible enough. I looked at the zstd-jni code and indeed it uses 
ZSTD_compressStream, ZSTD_finishStream, ZSTD_decompressStream in the same way 
as the Hadoop native codec when on the java side you use those stream classes. 
It would amount to implementing a new HBase codec. Call it ZstdStreamCodec 
maybe.

I could try that and see what happens, if the implementations could be read and 
write compatible?


was (Author: apurtell):
HBase native codecs all work by appending to a buffer and then compressing the 
whole buffer in one shot when the BlockCompressionStream calls the codec 
finish() method. This is how lz4 and snappy and some other codecs work both in 
Hadoop native and HBase implementations, and it was extended to all cases for 
simplicity. We use the Zstandard one shot API.
||BlockCompressionStream calls this||Codec does this||
|compress|buffer|
|compress|buffer|
|compress|buffer|
|finish|ZSTD_compress2 (via zstd-jni's Zstd#compress())|

One of the reasons we do that is aircompressor's limited zstandard support only 
operates in one-shot mode and does not offer a streaming API compatible with 
the native C zstandard library.

The hadoop native zstd codec uses the zstandard C library's streaming API, 
ZSTD_compressStream and ZSTD_decompressStream, basically wrapping the hierarchy 
of compression related Java stream APIs around this underlying native stream 
API.
||BlockCompressionStream calls this||Codec does this||
|compress|ZSTD_compressStream|
|compress|ZSTD_compressStream|
|compress|ZSTD_compressStream|
|finish|ZSTD_finishStream|

So what you get framed in the blocks of BlockCompressionStream is different 
enough to not be compatible.

In theory a new HBase native zstandard codec could do the same sequence of 
operations as Hadoop's native one by using zstd-jni's 
ZstdDirectBufferCompressingStream and ZstdDirectBufferDecompressingStream 
instead. I am not sure the behavior would be 100% identical but maybe 
compatible enough. I looked at the zstd-jni code and indeed it uses 
ZSTD_compressStream, ZSTD_finishStream, ZSTD_decompressStream in the same way 
as the Hadoop native codec. It would amount to implementing a new HBase codec. 
Call it ZstdStreamCodec maybe.

I could try that and see what happens, if the implementations could be read and 
write compatible?

> Possible Zstd incompatibility
> -----------------------------
>
>                 Key: HBASE-27706
>                 URL: https://issues.apache.org/jira/browse/HBASE-27706
>             Project: HBase
>          Issue Type: Bug
>          Components: compatibility
>    Affects Versions: 2.5.3
>            Reporter: Frens Jan Rumph
>            Priority: Major
>
>  
> We're in the process of upgrading a HBase installation from 2.2.4 to 2.5.3. 
> We're currently using Zstd compression from our Hadoop installation. Due to 
> some other class path issues (Netty issues in relation to the async WAL 
> provider), we would like to remove Hadoop from the class path.
> However, using the Zstd compression from HBase (which uses 
> [https://github.com/luben/zstd-jni]) we seem to hit some incompatibility. 
> When restarting a node to use this implementation we had errors like the 
> following:
> {code:java}
> 2023-03-10 16:33:01,925 WARN  [RS_OPEN_REGION-regionserver/n2:16020-0] 
> handler.AssignRegionHandler: Failed to open region 
> NAMESPACE:TABLE,,1673888962751.cdb726dad4eaabf765969f195e91c737., will report 
> to master
> java.io.IOException: java.io.IOException: 
> org.apache.hadoop.hbase.io.hfile.CorruptHFileException: Problem reading data 
> index and meta index from file 
> hdfs://CLUSTER/hbase/data/NAMESPACE/TABLE/cdb726dad4eaabf765969f195e91c737/e/aea6eddaa8ee476197d064a4b4c345b9
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.initializeStores(HRegion.java:1148)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.initializeStores(HRegion.java:1091)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.initializeRegionInternals(HRegion.java:994)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:941)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7228)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegionFromTableDir(HRegion.java:7183)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7159)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7118)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:7074)
>         at 
> org.apache.hadoop.hbase.regionserver.handler.AssignRegionHandler.process(AssignRegionHandler.java:147)
>         at 
> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:100)
>         at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>         at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>         at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: java.io.IOException: 
> org.apache.hadoop.hbase.io.hfile.CorruptHFileException: Problem reading data 
> index and meta index from file 
> hdfs://CLUSTER/hbase/data/NAMESPACE/TABLE/cdb726dad4eaabf765969f195e91c737/e/aea6eddaa8ee476197d064a4b4c345b9
>         at 
> org.apache.hadoop.hbase.regionserver.StoreEngine.openStoreFiles(StoreEngine.java:288)
>         at 
> org.apache.hadoop.hbase.regionserver.StoreEngine.initialize(StoreEngine.java:338)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:297)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:6359)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion$1.call(HRegion.java:1114)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion$1.call(HRegion.java:1111)
>         at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
>         at 
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
>         at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
>         ... 3 more
> Caused by: org.apache.hadoop.hbase.io.hfile.CorruptHFileException: Problem 
> reading data index and meta index from file 
> hdfs://CLUSTER/hbase/data/NAMESPACE/TABLE/cdb726dad4eaabf765969f195e91c737/e/aea6eddaa8ee476197d064a4b4c345b9
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileInfo.initMetaAndIndex(HFileInfo.java:392)
>         at 
> org.apache.hadoop.hbase.regionserver.HStoreFile.open(HStoreFile.java:394)
>         at 
> org.apache.hadoop.hbase.regionserver.HStoreFile.initReader(HStoreFile.java:518)
>         at 
> org.apache.hadoop.hbase.regionserver.StoreEngine.createStoreFileAndReader(StoreEngine.java:225)
>         at 
> org.apache.hadoop.hbase.regionserver.StoreEngine.lambda$openStoreFiles$0(StoreEngine.java:266)
>         ... 6 more
> Caused by: java.io.IOException: Premature EOF from inputStream, but still 
> need 2883 bytes
>         at 
> org.apache.hadoop.hbase.io.util.BlockIOUtils.readFullyWithHeapBuffer(BlockIOUtils.java:153)
>         at 
> org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext.prepareDecoding(HFileBlockDefaultDecodingContext.java:104)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock.unpack(HFileBlock.java:644)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl$1.nextBlock(HFileBlock.java:1397)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl$1.nextBlockWithBlockType(HFileBlock.java:1407)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileInfo.initMetaAndIndex(HFileInfo.java:365)
>         ... 10 more {code}
> I've been able to reproduce the issue with something like:
> {code:java}
> Configuration conf = HBaseConfiguration.create();
> conf.set("hbase.io.compress.zstd.codec", 
> "org.apache.hadoop.hbase.io.compress.zstd.ZstdCodec");
> HFileSystem fs = (HFileSystem) HFileSystem.get(conf);
> HFile.createReader(fs, new Path(...), conf); {code}
> with a file from HDFS that was created with the native compressor from Hadoop.
> Note that I only _suspect_ that this issue is caused by Zstd! In our test 
> environment we are already running 2.5.3 with reasonable succes. This issue 
> arises when we drop Hadoop from the class path and use the 'built in' 
> compression. But that's not hard evidence of Zstd being the root cause of 
> course.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to