virajjasani commented on a change in pull request #3691: URL: https://github.com/apache/hbase/pull/3691#discussion_r718516356
########## File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java ########## @@ -470,4 +551,41 @@ public static void decompress(ByteBuff dest, InputStream bufferedBoundedStream, } } } + + public static CompressionCodec buildCodec(final Configuration conf, final Algorithm algo) { + try { + String codecClassName = conf.get(algo.confKey, algo.confDefault); + if (codecClassName == null) { + throw new RuntimeException("No codec configured for " + algo.confKey); + } + Class<?> codecClass = getClassLoaderForCodec().loadClass(codecClassName); + CompressionCodec codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, + new Configuration(conf)); + LOG.info("Loaded codec {} for compression algorithm {}", + codec.getClass().getCanonicalName(), algo.name()); + return codec; + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public static void main(String[] args) throws Exception { Review comment: throws clause might not be required it seems ########## File path: hbase-common/src/test/java/org/apache/hadoop/hbase/io/compress/CompressionTestBase.java ########## @@ -0,0 +1,140 @@ +/* + * 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 static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.Arrays; +import java.util.Random; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.RandomDistribution; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CompressionTestBase { + + protected static final Logger LOG = LoggerFactory.getLogger(CompressionTestBase.class); + + static final int LARGE_SIZE = 10 * 1024 * 1024; + static final int VERY_LARGE_SIZE = 100 * 1024 * 1024; + static final int BLOCK_SIZE = 4096; + + static final byte[] SMALL_INPUT; + static { + // 1, 1, 2, 3, 5, 8, 13, 21, 34, 55, 89, 144, 233, 377, 610, 987, 1597 + SMALL_INPUT = new byte[1+1+2+3+5+8+13+21+34+55+89+144+233+377+610+987+1597]; + int off = 0; + Arrays.fill(SMALL_INPUT, off, (off+=1), (byte)'A'); + Arrays.fill(SMALL_INPUT, off, (off+=1), (byte)'B'); + Arrays.fill(SMALL_INPUT, off, (off+=2), (byte)'C'); + Arrays.fill(SMALL_INPUT, off, (off+=3), (byte)'D'); + Arrays.fill(SMALL_INPUT, off, (off+=5), (byte)'E'); + Arrays.fill(SMALL_INPUT, off, (off+=8), (byte)'F'); + Arrays.fill(SMALL_INPUT, off, (off+=13), (byte)'G'); + Arrays.fill(SMALL_INPUT, off, (off+=21), (byte)'H'); + Arrays.fill(SMALL_INPUT, off, (off+=34), (byte)'I'); + Arrays.fill(SMALL_INPUT, off, (off+=55), (byte)'J'); + Arrays.fill(SMALL_INPUT, off, (off+=89), (byte)'K'); + Arrays.fill(SMALL_INPUT, off, (off+=144), (byte)'L'); + Arrays.fill(SMALL_INPUT, off, (off+=233), (byte)'M'); + Arrays.fill(SMALL_INPUT, off, (off+=377), (byte)'N'); + Arrays.fill(SMALL_INPUT, off, (off+=610), (byte)'O'); + Arrays.fill(SMALL_INPUT, off, (off+=987), (byte)'P'); + Arrays.fill(SMALL_INPUT, off, (off+=1597), (byte)'Q'); + } + + protected void codecTest(final CompressionCodec codec, final byte[][] input) + throws Exception { + // We do this in Compression.java + ((Configurable)codec).getConf().setInt("io.file.buffer.size", 32 * 1024); + // Compress + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + CompressionOutputStream out = codec.createOutputStream(baos); + int inLen = 0; + long start = EnvironmentEdgeManager.currentTime(); + for (int i = 0; i < input.length; i++) { + out.write(input[i]); + inLen += input[i].length; + } Review comment: nit: could be replaced with enhanced loop? ``` for (byte[] in : input) { out.write(in); inLen += in.length; } ``` ########## File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java ########## @@ -470,4 +551,41 @@ public static void decompress(ByteBuff dest, InputStream bufferedBoundedStream, } } } + + public static CompressionCodec buildCodec(final Configuration conf, final Algorithm algo) { Review comment: Also, since this is common utility used by all codec reloads, good to provide one liner Javadoc. Nothing urgent, can be done at commit time too. ########## File path: hbase-compression/hbase-compression-aircompressor/src/main/java/org/apache/hadoop/hbase/io/compress/aircompressor/SnappyCodec.java ########## @@ -0,0 +1,144 @@ +/* + * 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.aircompressor; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import io.airlift.compress.snappy.SnappyCompressor; +import io.airlift.compress.snappy.SnappyDecompressor; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.io.compress.BlockCompressorStream; +import org.apache.hadoop.io.compress.BlockDecompressorStream; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Hadoop snappy codec implemented with aircompressor. + * <p> + * This is data format compatible with Hadoop's native snappy codec. + */ +@InterfaceAudience.Private +public class SnappyCodec implements Configurable, CompressionCodec { + + private Configuration conf; + + public SnappyCodec() { + conf = new Configuration(); + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Compressor createCompressor() { + return new HadoopSnappyCompressor(); + } + + @Override + public Decompressor createDecompressor() { + return new HadoopSnappyDecompressor(); + } + + @Override + public CompressionInputStream createInputStream(InputStream in) throws IOException { + return createInputStream(in, createDecompressor()); + } + + @Override + public CompressionInputStream createInputStream(InputStream in, Decompressor d) + throws IOException { + return new BlockDecompressorStream(in, d, getBufferSize(conf)); + } + + @Override + public CompressionOutputStream createOutputStream(OutputStream out) throws IOException { + return createOutputStream(out, createCompressor()); + } + + @Override + public CompressionOutputStream createOutputStream(OutputStream out, Compressor c) + throws IOException { + int bufferSize = getBufferSize(conf); + int compressionOverhead = (bufferSize / 6) + 32; + return new BlockCompressorStream(out, c, bufferSize, compressionOverhead); + } + + @Override + public Class<? extends Compressor> getCompressorType() { + return HadoopSnappyCompressor.class; + } + + @Override + public Class<? extends Decompressor> getDecompressorType() { + return HadoopSnappyDecompressor.class; + } + + @Override + public String getDefaultExtension() { + return ".snappy"; + } + + private static int getBufferSize(Configuration conf) { + int size = conf.getInt( + CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY, + CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_DEFAULT); + return size > 0 ? size : 256 * 1024; + } + + @InterfaceAudience.Private Review comment: Curious, is it necessary to define IA annotation for nested classes always? ########## File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java ########## @@ -470,4 +551,41 @@ public static void decompress(ByteBuff dest, InputStream bufferedBoundedStream, } } } + + public static CompressionCodec buildCodec(final Configuration conf, final Algorithm algo) { + try { + String codecClassName = conf.get(algo.confKey, algo.confDefault); + if (codecClassName == null) { + throw new RuntimeException("No codec configured for " + algo.confKey); + } + Class<?> codecClass = getClassLoaderForCodec().loadClass(codecClassName); + CompressionCodec codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, + new Configuration(conf)); + LOG.info("Loaded codec {} for compression algorithm {}", + codec.getClass().getCanonicalName(), algo.name()); + return codec; + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + java.util.Map<String, CompressionCodec> implMap = new java.util.HashMap<>(); + for (Algorithm algo: Algorithm.class.getEnumConstants()) { + try { + implMap.put(algo.name(), algo.getCodec(conf)); + } catch (Exception e) { } Review comment: catch block is redundant? ########## File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java ########## @@ -471,4 +471,57 @@ public static byte toByte(ByteBuffer buf, int offset) { public static byte toByte(Object ref, long offset) { return theUnsafe.getByte(ref, offset); } + + /** + * Zero fill a byte buffer as efficiently as possible. + * @param buf the byte buffer + */ + public static void clear(final ByteBuffer buf) { Review comment: Is this being used somewhere or this is for future purpose? ########## File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java ########## @@ -470,4 +551,41 @@ public static void decompress(ByteBuff dest, InputStream bufferedBoundedStream, } } } + + public static CompressionCodec buildCodec(final Configuration conf, final Algorithm algo) { Review comment: Could be `private`? ########## File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java ########## @@ -471,4 +471,57 @@ public static byte toByte(ByteBuffer buf, int offset) { public static byte toByte(Object ref, long offset) { return theUnsafe.getByte(ref, offset); } + + /** + * Zero fill a byte buffer as efficiently as possible. + * @param buf the byte buffer + */ + public static void clear(final ByteBuffer buf) { + if (buf.isDirect()) { + clear(((DirectBuffer)buf).address(), buf.capacity()); + } else { + clear(buf.array(), buf.arrayOffset(), buf.capacity()); + } + } + + /** + * Zero fill an array as efficiently as possible. + * @param o the array, as object + * @param offset starting offset for zero fill + * @param len length of zero fill + */ + public static void clear(Object o, final int offset, final int len) { + long off = theUnsafe.arrayBaseOffset(o.getClass()) + offset, + remaining = (long)len * theUnsafe.arrayIndexScale(o.getClass()); Review comment: Got bit confused the first time, this would improve readability I think ``` long off = theUnsafe.arrayBaseOffset(o.getClass()) + offset; long remaining = (long) len * theUnsafe.arrayIndexScale(o.getClass()); ``` -- 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: issues-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org