rmdmattingly commented on code in PR #6708: URL: https://github.com/apache/hbase/pull/6708#discussion_r1968025648
########## hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/BlockDecompressorHelper.java: ########## @@ -0,0 +1,67 @@ +/* + * 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); + compressedBytesConsumed += compressedChunkSize; + decompressedBytesInBlock += n; + totalDecompressedBytes += n; + } Review Comment: should we have some sort of check to bail out of the loop if RawDecompressor#decompress returns zero for some reason? Otherwise I think this logic would be stuck ########## hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/ByteBuffDecompressionCodec.java: ########## @@ -0,0 +1,31 @@ +/* + * 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 org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + [email protected] Review Comment: What's the motivation for making these new classes public? I wonder whether private or limited private (with config exposure) is more appropriate ########## hbase-compression/hbase-compression-zstd/src/main/java/org/apache/hadoop/hbase/io/compress/zstd/ZstdByteBuffDecompressor.java: ########## @@ -0,0 +1,143 @@ +/* + * 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; + protected ZstdDictDecompress dict; + protected ZstdDecompressCtx ctx; + + ZstdByteBuffDecompressor(@Nullable byte[] dictionary) { + ctx = new ZstdDecompressCtx(); + if (dictionary != null) { + this.dictId = ZstdCodec.getDictionaryId(dictionary); + this.dict = new ZstdDictDecompress(dictionary); + this.ctx.loadDict(this.dict); + } + } + + @Override + public boolean canDecompress(ByteBuff output, ByteBuff input) { + if (output instanceof SingleByteBuff && input instanceof SingleByteBuff) { + ByteBuffer nioOutput = output.nioByteBuffers()[0]; + ByteBuffer nioInput = input.nioByteBuffers()[0]; + if (nioOutput.isDirect() && nioInput.isDirect()) { + return true; + } else if (!nioOutput.isDirect() && !nioInput.isDirect()) { + return true; + } + } + + return false; + } + + @Override + public int decompress(ByteBuff output, ByteBuff input, int inputLen) throws IOException { + return BlockDecompressorHelper.decompress(output, input, inputLen, this::decompressRaw); + } + + private int decompressRaw(ByteBuff output, ByteBuff input, int inputLen) throws IOException { + if (output instanceof SingleByteBuff && input instanceof SingleByteBuff) { + ByteBuffer nioOutput = output.nioByteBuffers()[0]; + ByteBuffer nioInput = input.nioByteBuffers()[0]; + if (nioOutput.isDirect() && nioInput.isDirect()) { + return decompressDirectByteBuffers(nioOutput, nioInput, inputLen); + } else if (!nioOutput.isDirect() && !nioInput.isDirect()) { + return decompressHeapByteBuffers(nioOutput, nioInput, inputLen); + } + } + + throw new IllegalStateException("One buffer is direct and the other is not, " + + "or one or more not SingleByteBuffs. This is not supported"); + } + + private int decompressDirectByteBuffers(ByteBuffer output, ByteBuffer input, int inputLen) { + int origOutputPos = output.position(); + + int n = ctx.decompressDirectByteBuffer(output, output.position(), + output.limit() - output.position(), input, input.position(), inputLen); + + output.position(origOutputPos + n); + return n; + } + + private int decompressHeapByteBuffers(ByteBuffer output, ByteBuffer input, int inputLen) { + int origOutputPos = output.position(); + + int n = ctx.decompressByteArray(output.array(), output.arrayOffset() + output.position(), + output.limit() - output.position(), input.array(), input.arrayOffset() + input.position(), + inputLen); + + output.position(origOutputPos + n); + return n; + } + + @Override + public void close() { + ctx.close(); + dict.close(); Review Comment: I think it's possible for this to produce a NPE -- 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]
