You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2022/04/19 13:12:38 UTC

[GitHub] [hbase] ndimiduk commented on a diff in pull request #4353: HBASE-26959 Brotli compression support

ndimiduk commented on code in PR #4353:
URL: https://github.com/apache/hbase/pull/4353#discussion_r853026481


##########
hbase-compression/hbase-compression-brotli/src/main/java/org/apache/hadoop/hbase/io/compress/brotli/BrotliCodec.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.brotli;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+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 brotli codec implemented with Brotli4j
+ */
+@InterfaceAudience.Private
+public class BrotliCodec implements Configurable, CompressionCodec {
+
+  public static final String BROTLI_LEVEL_KEY = "hbase.io.compress.brotli.level";
+  // Our default is 6, based on https://blog.cloudflare.com/results-experimenting-brotli/
+  public static final int BROTLI_LEVEL_DEFAULT = 6; // [0,11] or -1
+  public static final String BROTLI_WINDOW_KEY = "hbase.io.compress.brotli.window";
+  public static final int BROTLI_WINDOW_DEFAULT = -1; // [10-24] or -1
+  public static final String BROTLI_BUFFERSIZE_KEY = "hbase.io.compress.brotli.buffersize";
+  public static final int BROTLI_BUFFERSIZE_DEFAULT = 256 * 1024;
+
+  private Configuration conf;
+
+  public BrotliCodec() {
+    conf = new Configuration();
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Compressor createCompressor() {
+    return new BrotliCompressor(getLevel(conf), getWindow(conf), getBufferSize(conf));
+  }
+
+  @Override
+  public Decompressor createDecompressor() {
+    return new BrotliDecompressor(getBufferSize(conf));
+  }
+
+  @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 BrotliCompressor.class;
+  }
+
+  @Override
+  public Class<? extends Decompressor> getDecompressorType() {
+    return BrotliDecompressor.class;
+  }
+
+  @Override
+  public String getDefaultExtension() {
+    return ".br";
+  }
+
+  // Package private
+
+  static int getLevel(Configuration conf) {
+    return conf.getInt(BROTLI_LEVEL_KEY, BROTLI_LEVEL_DEFAULT);
+  }
+
+  static int getWindow(Configuration conf) {
+    return conf.getInt(BROTLI_WINDOW_KEY, BROTLI_WINDOW_DEFAULT);
+  }
+
+  static int getBufferSize(Configuration conf) {
+    int size = conf.getInt(BROTLI_BUFFERSIZE_KEY, BROTLI_BUFFERSIZE_DEFAULT);
+    return size > 0 ? size : 256 * 1024; // Don't change this default

Review Comment:
   Should we reuse the constant `BROTLI_BUFFERSIZE_DEFAULT` ?



##########
hbase-compression/hbase-compression-brotli/pom.xml:
##########
@@ -0,0 +1,167 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+/**
+ * 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.
+ */
+-->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase-compression</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>3.0.0-alpha-3-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>hbase-compression-brotli</artifactId>
+  <name>Apache HBase - Compression - Brotli</name>
+  <description>Compression support using Brotli4j</description>
+  <build>
+    <plugins>
+      <!-- Testing plugins -->
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>net.revelc.code</groupId>
+        <artifactId>warbucks-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <!--Make it so assembly:single does nothing in here-->
+          <artifactId>maven-assembly-plugin</artifactId>
+          <configuration>
+            <skipAssembly>true</skipAssembly>
+          </configuration>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-checkstyle-plugin</artifactId>
+          <configuration>
+            <failOnViolation>true</failOnViolation>
+          </configuration>
+        </plugin>
+        <plugin>
+          <groupId>net.revelc.code</groupId>
+          <artifactId>warbucks-maven-plugin</artifactId>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+  <dependencies>
+    <!-- Intra-project dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-logging</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.github.stephenc.findbugs</groupId>
+      <artifactId>findbugs-annotations</artifactId>
+      <scope>compile</scope>
+      <optional>true</optional>
+    </dependency>
+    <!-- native Java compression codecs -->
+    <dependency>
+      <groupId>com.aayushatharva.brotli4j</groupId>

Review Comment:
   Should we shade this dependency? Is there a scenario where user code and this jar find themselves on the same class path?



##########
hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm:
##########
@@ -1381,7 +1381,7 @@ You can redistribute it and/or modify it under either the terms of the
 ## See this FAQ link for justifications: https://www.apache.org/legal/resolved.html
 ##
 ## NB: This list is later compared as lower-case. New entries must also be all lower-case
-#set($non_aggregate_fine = [ 'public domain', 'new bsd license', 'bsd license', 'bsd', 'bsd 2-clause license', 'mozilla public license version 1.1', 'mozilla public license version 2.0', 'creative commons attribution license, version 2.5' ])
+#set($non_aggregate_fine = [ 'public domain', 'new bsd license', 'bsd license', 'bsd', 'bsd 2-clause license', 'mozilla public license version 1.1', 'mozilla public license version 2.0', 'creative commons attribution license, version 2.5', 'apache license 2.0' ])

Review Comment:
   I'm surprised we need this change.
   
   @busbey ?



##########
hbase-compression/hbase-compression-brotli/src/main/java/org/apache/hadoop/hbase/io/compress/brotli/BrotliDecompressor.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.brotli;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.Decoder;
+import com.aayushatharva.brotli4j.decoder.Decoders;

Review Comment:
   Unused import.



##########
hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java:
##########
@@ -352,6 +358,30 @@ public CompressionCodec reload(Configuration conf) {
           return lzmaCodec;
         }
       }
+    },
+    BROTLI("brotli", BROTLI_CODEC_CLASS_KEY, BROTLI_CODEC_CLASS_DEFAULT) {

Review Comment:
   Consider adding `@SuppressWarnings("ImmutableEnumChecker")` -- I think this will quiet ErrorProne related to this lazy initializer.



##########
hbase-compression/hbase-compression-brotli/src/main/java/org/apache/hadoop/hbase/io/compress/brotli/BrotliCodec.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.brotli;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+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 brotli codec implemented with Brotli4j
+ */
+@InterfaceAudience.Private
+public class BrotliCodec implements Configurable, CompressionCodec {
+
+  public static final String BROTLI_LEVEL_KEY = "hbase.io.compress.brotli.level";
+  // Our default is 6, based on https://blog.cloudflare.com/results-experimenting-brotli/
+  public static final int BROTLI_LEVEL_DEFAULT = 6; // [0,11] or -1
+  public static final String BROTLI_WINDOW_KEY = "hbase.io.compress.brotli.window";
+  public static final int BROTLI_WINDOW_DEFAULT = -1; // [10-24] or -1
+  public static final String BROTLI_BUFFERSIZE_KEY = "hbase.io.compress.brotli.buffersize";
+  public static final int BROTLI_BUFFERSIZE_DEFAULT = 256 * 1024;
+
+  private Configuration conf;
+
+  public BrotliCodec() {
+    conf = new Configuration();
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Compressor createCompressor() {
+    return new BrotliCompressor(getLevel(conf), getWindow(conf), getBufferSize(conf));
+  }
+
+  @Override
+  public Decompressor createDecompressor() {
+    return new BrotliDecompressor(getBufferSize(conf));
+  }
+
+  @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;

Review Comment:
   What is this magic number? Should this be the `Math.min` of this and some value? Can you add a comment that documents the reasoning behind the formula used here?



##########
hbase-compression/hbase-compression-brotli/src/main/java/org/apache/hadoop/hbase/io/compress/brotli/BrotliCompressor.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.brotli;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import com.aayushatharva.brotli4j.encoder.Encoders;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.compress.CanReinit;
+import org.apache.hadoop.hbase.io.compress.CompressionUtil;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Hadoop compressor glue for Brotli4j
+ */
+@InterfaceAudience.Private
+public class BrotliCompressor implements CanReinit, Compressor {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(BrotliCompressor.class);
+  protected ByteBuffer inBuf, outBuf;
+  protected int bufferSize;
+  protected boolean finish, finished;
+  protected long bytesRead, bytesWritten;
+  protected Encoder.Parameters params;
+
+  static {
+    Brotli4jLoader.ensureAvailability();
+  }
+
+  BrotliCompressor(int level, int window, int bufferSize) {
+    this.bufferSize = bufferSize;
+    this.inBuf = ByteBuffer.allocate(bufferSize);
+    this.outBuf = ByteBuffer.allocate(bufferSize);
+    this.outBuf.position(bufferSize);
+    params = new Encoder.Parameters();
+    params.setQuality(level);
+    params.setWindow(window);
+  }
+
+  @Override
+  public int compress(byte[] b, int off, int len) throws IOException {
+    // If we have previously compressed our input and still have some buffered bytes
+    // remaining, provide them to the caller.
+    if (outBuf.hasRemaining()) {
+      int remaining = outBuf.remaining(), n = Math.min(remaining, len);
+      outBuf.get(b, off, n);
+      LOG.trace("compress: {} bytes from outBuf", n);

Review Comment:
   nit: "_read_ n _remaining_ bytes from outBuf".



##########
hbase-compression/hbase-compression-brotli/src/main/java/org/apache/hadoop/hbase/io/compress/brotli/BrotliCompressor.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.brotli;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import com.aayushatharva.brotli4j.encoder.Encoders;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.compress.CanReinit;
+import org.apache.hadoop.hbase.io.compress.CompressionUtil;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Hadoop compressor glue for Brotli4j
+ */
+@InterfaceAudience.Private
+public class BrotliCompressor implements CanReinit, Compressor {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(BrotliCompressor.class);
+  protected ByteBuffer inBuf, outBuf;
+  protected int bufferSize;
+  protected boolean finish, finished;
+  protected long bytesRead, bytesWritten;
+  protected Encoder.Parameters params;
+
+  static {
+    Brotli4jLoader.ensureAvailability();
+  }
+
+  BrotliCompressor(int level, int window, int bufferSize) {
+    this.bufferSize = bufferSize;
+    this.inBuf = ByteBuffer.allocate(bufferSize);
+    this.outBuf = ByteBuffer.allocate(bufferSize);
+    this.outBuf.position(bufferSize);
+    params = new Encoder.Parameters();
+    params.setQuality(level);
+    params.setWindow(window);
+  }
+
+  @Override
+  public int compress(byte[] b, int off, int len) throws IOException {
+    // If we have previously compressed our input and still have some buffered bytes
+    // remaining, provide them to the caller.
+    if (outBuf.hasRemaining()) {
+      int remaining = outBuf.remaining(), n = Math.min(remaining, len);
+      outBuf.get(b, off, n);
+      LOG.trace("compress: {} bytes from outBuf", n);
+      return n;
+    }
+    // We don't actually begin compression until our caller calls finish().
+    if (finish) {
+      if (inBuf.position() > 0) {
+        inBuf.flip();
+        int uncompressed = inBuf.remaining();
+        // If we don't have enough capacity in our currently allocated output buffer,
+        // allocate a new one which does.
+        int needed = maxCompressedLength(uncompressed);
+        // Can we compress directly into the provided array?
+        boolean direct = false;
+        ByteBuffer writeBuf;
+        if (len <= needed) {
+          direct = true;
+          writeBuf = ByteBuffer.wrap(b, off, len);
+        } else {
+          if (outBuf.capacity() < needed) {
+            needed = CompressionUtil.roundInt2(needed);
+            LOG.trace("compress: resize outBuf {}", needed);
+            outBuf = ByteBuffer.allocate(needed);
+          } else {
+            outBuf.clear();
+          }
+          writeBuf = outBuf;
+        }
+        final int oldPos = writeBuf.position();
+        Encoders.compress(inBuf, writeBuf, params);
+        final int written = writeBuf.position() - oldPos;
+        bytesWritten += written;
+        inBuf.clear();
+        LOG.trace("compress: compressed {} -> {}", uncompressed, written);
+        finished = true;
+        if (!direct) {
+          outBuf.flip();
+          int n = Math.min(written, len);
+          outBuf.get(b, off, n);
+          LOG.trace("compress: {} bytes", n);
+          return n;
+        } else {
+          LOG.trace("compress: {} bytes direct", written);
+          return written;
+        }
+      } else {
+        finished = true;
+      }
+    }
+    LOG.trace("No output");
+    return 0;
+  }
+
+  @Override
+  public void end() {
+    LOG.trace("end");
+  }
+
+  @Override
+  public void finish() {
+    LOG.trace("finish");
+    finish = true;
+  }
+
+  @Override
+  public boolean finished() {
+    boolean b = finished && !outBuf.hasRemaining();
+    LOG.trace("finished: {}", b);
+    return b;
+  }
+
+  @Override
+  public long getBytesRead() {
+    return bytesRead;
+  }
+
+  @Override
+  public long getBytesWritten() {
+    return bytesWritten;
+  }
+
+  @Override
+  public boolean needsInput() {
+    boolean b = !finished();
+    LOG.trace("needsInput: {}", b);
+    return b;
+  }
+
+  @Override
+  public void reinit(Configuration conf) {
+    LOG.trace("reinit");
+    if (conf != null) {
+      // Quality or window settings might have changed
+      params.setQuality(BrotliCodec.getLevel(conf));
+      params.setWindow(BrotliCodec.getWindow(conf));
+      // Buffer size might have changed
+      int newBufferSize = BrotliCodec.getBufferSize(conf);
+      if (bufferSize != newBufferSize) {
+        bufferSize = newBufferSize;
+        this.inBuf = ByteBuffer.allocateDirect(bufferSize);
+        this.outBuf = ByteBuffer.allocateDirect(bufferSize);
+      }
+    }
+    reset();
+  }
+
+  @Override
+  public void reset() {
+    LOG.trace("reset");
+    inBuf.clear();
+    outBuf.clear();
+    outBuf.position(outBuf.capacity());
+    bytesRead = 0;
+    bytesWritten = 0;
+    finish = false;
+    finished = false;
+  }
+
+  @Override
+  public void setDictionary(byte[] b, int off, int len) {
+    // TODO: Brotli4j supports custom dictionaries, but this usage is not expected
+    throw new UnsupportedOperationException("setDictionary is not supported");
+  }
+
+  @Override
+  public void setInput(byte[] b, int off, int len) {
+    LOG.trace("setInput: off={} len={}", off, len);
+    if (inBuf.remaining() < len) {
+      // Get a new buffer that can accomodate the accumulated input plus the additional
+      // input that would cause a buffer overflow without reallocation.
+      // This condition should be fortunately rare, because it is expensive.
+      int needed = CompressionUtil.roundInt2(inBuf.capacity() + len);
+      LOG.trace("setInput: resize inBuf {}", needed);
+      ByteBuffer newBuf = ByteBuffer.allocate(needed);
+      inBuf.flip();
+      newBuf.put(inBuf);
+      inBuf = newBuf;
+    }
+    inBuf.put(b, off, len);
+    bytesRead += len;
+    finished = false;
+  }
+
+  // Package private
+
+  int maxCompressedLength(int len) {
+    return len + 32 + (len/6);

Review Comment:
   Same questions as from the Codec.



-- 
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-unsubscribe@hbase.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org