ndimiduk commented on code in PR #6708: URL: https://github.com/apache/hbase/pull/6708#discussion_r1969783344
########## hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/BlockDecompressorHelper.java: ########## @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.compress; + +import java.io.IOException; +import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.yetus.audience.InterfaceAudience; + [email protected] +public class BlockDecompressorHelper { + + public interface RawDecompressor { + int decompress(ByteBuff output, ByteBuff input, int inputLen) throws IOException; + } + + /** + * Helper to decompress a ByteBuff that was created by a + * {@link org.apache.hadoop.io.compress.BlockCompressorStream}, or is at least in the same format. + * Parses the binary format and delegates actual decompression work to the provided + * {@link RawDecompressor}. + */ + public static int decompress(ByteBuff output, ByteBuff input, int inputSize, + RawDecompressor rawDecompressor) throws IOException { + int totalDecompressedBytes = 0; + int compressedBytesConsumed = 0; + + while (compressedBytesConsumed < inputSize) { + int decompressedBlockSize = rawReadInt(input); + compressedBytesConsumed += 4; + int decompressedBytesInBlock = 0; + + while (decompressedBytesInBlock < decompressedBlockSize) { + int compressedChunkSize = rawReadInt(input); + compressedBytesConsumed += 4; + int n = rawDecompressor.decompress(output, input, compressedChunkSize); + if (n <= 0) { + throw new IOException("Decompression failed. Compressed size: " + compressedChunkSize + + ", decompressed size: " + decompressedBlockSize); + } + compressedBytesConsumed += compressedChunkSize; + decompressedBytesInBlock += n; + totalDecompressedBytes += n; + } + } + return totalDecompressedBytes; + } + + private static int rawReadInt(ByteBuff input) { Review Comment: I think that you don't need to implement this method. Instead, call `ByteBuff#getInt()`. It uses the Unsafe to read the full 4 bytes at once. ########## hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/ByteBuffDecompressor.java: ########## @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.compress; + +import java.io.Closeable; +import java.io.IOException; +import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Specification of a block-based decompressor, which can be more efficient than the stream-based Review Comment: nit: is it "block-based", or "ByteBuff-based"? Nothing in the interface name or methods tells me that it's only decompressing a single serialised `HFileBlock`s. Does it operate on a single block at a time, or can I provide it an `inputLen` that represents several blocks in the same input buffer? Maybe all this is sort of assumed by the existing conventions in this package. ########## hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java: ########## @@ -119,6 +128,49 @@ public void prepareDecoding(int onDiskSizeWithoutHeader, int uncompressedSizeWit } } + /** + * When only decompression is needed (not decryption), and the input and output buffers are + * SingleByteBuffs, and the decompression algorithm supports it, we can do decompression without + * any intermediate heap buffers. Do not call unless you've checked {@link #canFastDecompress} + * first. + */ + private void fastDecompress(ByteBuff blockBufferWithoutHeader, ByteBuff onDiskBlock, + int onDiskSizeWithoutHeader) throws IOException { + Compression.Algorithm compression = fileContext.getCompression(); + ByteBuffDecompressor decompressor = compression.getByteBuffDecompressor(); + try { + if (decompressor instanceof CanReinit) { + ((CanReinit) decompressor).reinit(conf); + } + decompressor.decompress(blockBufferWithoutHeader, onDiskBlock, onDiskSizeWithoutHeader); + } finally { + compression.returnByteBuffDecompressor(decompressor); + } + } + + private boolean canFastDecompress(ByteBuff blockBufferWithoutHeader, ByteBuff onDiskBlock) { Review Comment: "fast" is relative and will likely continue to change. Instead, can you use a more descriptive name for this alternative implementation. Maybe `canDecompressViaByteBuffDecompressor`? ########## hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java: ########## @@ -507,6 +508,46 @@ public void returnDecompressor(Decompressor decompressor) { } } + /** + * Signals if this codec theoretically supports decompression on {@link ByteBuff}s. This can be + * faster than using a DecompressionStream. If this method returns true, you can call + * {@link #getByteBuffDecompressor()} to obtain a {@link ByteBuffDecompressor}. You must then + * also call {@link ByteBuffDecompressor#canDecompress(ByteBuff, ByteBuff)} before attempting + * decompression, to verify if that decompressor is capable of handling your particular input + * and output buffers. + */ + public boolean supportsByteBuffDecompression() { + CompressionCodec codec = getCodec(conf); + return codec instanceof ByteBuffDecompressionCodec; + } + + /** + * Be sure to call {@link #supportsByteBuffDecompression()} before calling this method. + * @throws IllegalStateException if the codec does not support block decompression + */ + public ByteBuffDecompressor getByteBuffDecompressor() { + CompressionCodec codec = getCodec(conf); + if (codec instanceof ByteBuffDecompressionCodec) { + ByteBuffDecompressor decompressor = + CodecPool.getByteBuffDecompressor((ByteBuffDecompressionCodec) codec); + if (LOG.isTraceEnabled()) { + LOG.trace("Retrieved decompressor " + decompressor + " from pool."); Review Comment: nit: use Logger format string API instead. ########## hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java: ########## @@ -507,6 +508,46 @@ public void returnDecompressor(Decompressor decompressor) { } } + /** + * Signals if this codec theoretically supports decompression on {@link ByteBuff}s. This can be + * faster than using a DecompressionStream. If this method returns true, you can call + * {@link #getByteBuffDecompressor()} to obtain a {@link ByteBuffDecompressor}. You must then + * also call {@link ByteBuffDecompressor#canDecompress(ByteBuff, ByteBuff)} before attempting + * decompression, to verify if that decompressor is capable of handling your particular input + * and output buffers. + */ + public boolean supportsByteBuffDecompression() { + CompressionCodec codec = getCodec(conf); + return codec instanceof ByteBuffDecompressionCodec; + } + + /** + * Be sure to call {@link #supportsByteBuffDecompression()} before calling this method. + * @throws IllegalStateException if the codec does not support block decompression + */ + public ByteBuffDecompressor getByteBuffDecompressor() { + CompressionCodec codec = getCodec(conf); + if (codec instanceof ByteBuffDecompressionCodec) { + ByteBuffDecompressor decompressor = + CodecPool.getByteBuffDecompressor((ByteBuffDecompressionCodec) codec); + if (LOG.isTraceEnabled()) { + LOG.trace("Retrieved decompressor " + decompressor + " from pool."); + } + return decompressor; + } else { + throw new IllegalStateException("Codec " + codec + " does not support block decompression"); + } + } + + public void returnByteBuffDecompressor(ByteBuffDecompressor decompressor) { + if (decompressor != null) { + if (LOG.isTraceEnabled()) { + LOG.trace("Returning decompressor " + decompressor + " to pool."); Review Comment: And here. ########## hbase-compression/hbase-compression-zstd/src/main/java/org/apache/hadoop/hbase/io/compress/zstd/ZstdByteBuffDecompressor.java: ########## @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.compress.zstd; + +import com.github.luben.zstd.ZstdDecompressCtx; +import com.github.luben.zstd.ZstdDictDecompress; +import edu.umd.cs.findbugs.annotations.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.io.compress.BlockDecompressorHelper; +import org.apache.hadoop.hbase.io.compress.ByteBuffDecompressor; +import org.apache.hadoop.hbase.io.compress.CanReinit; +import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.hadoop.hbase.nio.SingleByteBuff; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Glue for ByteBuffDecompressor on top of zstd-jni + */ [email protected] +public class ZstdByteBuffDecompressor implements ByteBuffDecompressor, CanReinit { + + protected int dictId; Review Comment: Why are these fields protected instead of private? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
