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 2021/05/08 01:34:58 UTC

[GitHub] [hbase] apurtell opened a new pull request #3244: [WIP] HBASE-25869 WAL value compression

apurtell opened a new pull request #3244:
URL: https://github.com/apache/hbase/pull/3244


   WAL storage can be expensive, especially if the cell values
   represented in the edits are large, consisting of blobs or
   significant lengths of text. Such WALs might need to be kept around
   for a fairly long time to satisfy replication constraints on a space
   limited (or space-contended) filesystem.
   
   We have a custom dictionary compression scheme for cell metadata that
   is engaged when WAL compression is enabled in site configuration. This
   is fine for that application, where we can expect the universe of values
   and their lengths in the custom dictionaries to be constrained. For
   arbitrary cell values it is better to use Deflate compression, which is
   which is a complete LZ-class algorithm suitable for arbitrary albeit
   compressible data, is reasonably fast, certainly fast enough for WALs,
   compresses well, and is universally available as part of the Java
   runtime.
   
   With a trick that encodes whether or not the cell value is compressed in
   the high order bit of the type byte, this can be done in a backwards
   compatible manner.


-- 
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.

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



[GitHub] [hbase] shahrs87 commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
shahrs87 commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-838959572


   Speaking out loud. During RU, the wal written by RS that has `ENABLE_WAL_VALUE_COMPRESSION` set to `false` died and another RS which was recently upgraded has the above property set to true. Will the new RS be able to decode the cell ?


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630657193



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize];
+      // Deflater#deflate will return 0 only if more input is required. We iterate until
+      // that condition is met, sending the content of 'buffer' to the output stream at
+      // each step, until deflate returns 0. Then the compressor must be flushed in order
+      // for all of the value's output to be written into the corresponding edit. (Otherwise
+      // the compressor would carry over some of the output for this value into the output
+      // of the next.) To flush the compressor we call deflate again using the method option
+      // that allows us to specify the SYNC_FLUSH flag. The sync output will be placed into
+      // the buffer. When flushing we iterate until there is no more output. Then the flush
+      // is complete and the compressor is ready for more input.
+      int bytesOut;

Review comment:
       With the latest code this won't let us reuse the buffer needed for Deflator#deflate. Maybe not important. I can look at it.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629779580



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       Sorry, these need to be public, at least the one, then might as well do both...
   
       hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java:[42,37]
       ENABLE_WAL_VALUE_COMPRESSION is not public in org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
       cannot be accessed from outside package




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634838266



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -70,18 +150,55 @@ public CompressionContext(Class<? extends Dictionary> dictType, boolean recovere
     if (hasTagCompression) {
       tagCompressionContext = new TagCompressionContext(dictType, Short.MAX_VALUE);
     }
+    if (hasValueCompression && valueCompressionType != null) {
+      valueCompressor = new ValueCompressor(valueCompressionType);
+    }
+  }
+
+  public CompressionContext(Class<? extends Dictionary> dictType, boolean recoveredEdits,
+      boolean hasTagCompression)
+      throws SecurityException, NoSuchMethodException, InstantiationException,
+        IllegalAccessException, InvocationTargetException, IOException {
+    this(dictType, recoveredEdits, hasTagCompression, false, null);
+  }
+
+  public boolean hasTagCompression() {
+    return tagCompressionContext != null;
+  }
+
+  public boolean hasValueCompression() {
+    return valueCompressor != null;
   }
 
-  public Dictionary getDictionary(Enum dictIndex) {
+  public Dictionary getDictionary(Enum<DictionaryIndex> dictIndex) {

Review comment:
       Let me just leave this alone, the same as before I touched this file. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839507875


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 33s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 50s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 36s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 37s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 40s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  4s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 11s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  | 144m 22s |  hbase-server in the patch failed.  |
   |  |   | 185m 51s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d3fda0b99f98 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/testReport/ |
   | Max. process+thread count | 3609 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r628663543



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -230,6 +230,11 @@ public static long getKeyDataStructureSize(int rlength, int flength, int qlength
     DeleteColumn((byte)12),
     DeleteFamily((byte)14),
 
+    // Effective maximum is 127 (Byte.MAX_VALUE). We set the high order bit of the
+    // type byte in the WAL codecs to indicate, in a backwards compatible way, if the
+    // value is compressed there.
+    EffectiveMaximum((byte)Byte.MAX_VALUE),

Review comment:
       This is the one ugly thing about the trick I use to encode whether or not a value is compressed in a fully backwards compatible way. I'm not sure if anything need be done with `Maximum` below. It is no longer a valid value. Because we overload the high order bit the max KV type value is 127 after this change. 
   
   Of course, as I am proposing this change, I think it is fine. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634845897



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));
+        compressedIn = algorithm.createDecompressionStream(lowerIn, algorithm.getDecompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerIn.setDelegate(new ByteArrayInputStream(inBuffer));
+      }
+      return compressedIn.read(outArray, outOffset, outLength);

Review comment:
       I added an assertion to this effect in WALCellCodec#readCompressedValue

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));
+        compressedIn = algorithm.createDecompressionStream(lowerIn, algorithm.getDecompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerIn.setDelegate(new ByteArrayInputStream(inBuffer));
+      }
+      return compressedIn.read(outArray, outOffset, outLength);

Review comment:
       I added an assertion to this effect in `WALCellCodec#readCompressedValue`




-- 
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.

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634856523



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       Maybe you could just extend it and make a public constructor? The `in` field is protected so you can introduce get and set method in sub classes to access it. This will make your DelegatingInputStream smaller.




-- 
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.

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634396897



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       What is the advantage comparing to FilterInputStream? We can set the delegated stream?




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630662501



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -302,18 +367,28 @@ protected Cell parseCell() throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
       pos += elemLen;
 
-      // timestamp, type and value
-      int tsTypeValLen = length - pos;
+      // timestamp
+      long ts = StreamUtils.readLong(in);
+      pos = Bytes.putLong(backingArray, pos, ts);
+      // type and value
+      int typeValLen = length - pos;
       if (tagsLength > 0) {
-        tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+        typeValLen = typeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+      }
+      byte type = (byte)in.read();
+      pos = Bytes.putByte(backingArray, pos, type);
+      int valLen = typeValLen - 1;
+      if (compression.hasValueCompression()) {

Review comment:
       Yes. 
   
   Also applies to the check for if tag compression is enabled as well.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629779580



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       Sorry, these need to be public. 
   
       hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java:[42,37] ENABLE_WAL_VALUE_COMPRESSION is not public in org.apache.hadoop.hbase.regionserver.wal.CompressionContext; cannot be accessed from outside package

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       Sorry, these need to be public. 
   
       hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java:[42,37]
    ENABLE_WAL_VALUE_COMPRESSION is not public in org.apache.hadoop.hbase.regionserver.wal.CompressionContext; cannot be accessed from outside package

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       Sorry, these need to be public. 
   
       hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java:[42,37]
       ENABLE_WAL_VALUE_COMPRESSION is not public in org.apache.hadoop.hbase.regionserver.wal.CompressionContext; cannot be accessed from outside package




-- 
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.

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



[GitHub] [hbase] ndimiduk commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r635657600



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 

Review comment:
       Looks like there's a checkstyle plugin for Eclipse too, but I cannot tell from their site if it provides import ordering according to checkstyle configuration.




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-843781271


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   6m 13s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  6s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 15s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  0s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  9s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 57s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  7s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 50s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  | 218m 47s |  hbase-server in the patch failed.  |
   |  |   | 263m 26s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 82ad45ebc2eb 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 741b4b4674 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/testReport/ |
   | Max. process+thread count | 2716 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634828768



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {
+
+  InputStream lowerStream;

Review comment:
       I'm not anticipating multithreaded use of this class but I did put it into common. Let me document this more and use atomic field accessors to future proof. 




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush at the end of every cell to ensure each cell record persists all of the value data into the expected place. We have to do that anyway... Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to builds its dictionary across the whole file's contents, achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629763212



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -302,14 +343,28 @@ protected Cell parseCell() throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
       pos += elemLen;
 
-      // timestamp, type and value
-      int tsTypeValLen = length - pos;
+      // timestamp
+      long ts = StreamUtils.readLong(in);
+      pos = Bytes.putLong(backingArray, pos, ts);
+      // type and value
+      int typeValLen = length - pos;
       if (tagsLength > 0) {
-        tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+        typeValLen = typeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+      }
+      // high bit of type byte is 1 if value is compressed
+      byte type = (byte)in.read();
+      if ((type & 0x80) == 0x80) {

Review comment:
       We decide at each value whether to store it compressed or not. First, there is a threshold. Too small and there is no point. Even so, sometimes values will not compress, even though value compression is enabled and by size it is eligible. If this happens we throw the compressed version away and just store the original. 
   
   >  Isn't it better to just zlib compress all the values? WDYT.
   
   Some value data will not compress but we won't know ahead of time. 
   
   I'm not strongly wedded to this idea but it seems good. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845529048


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 55s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 50s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 40s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 27s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 53s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 53s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 49s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  5s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  8s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 140m 14s |  hbase-server in the patch passed.  |
   |  |   | 187m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2c288e42a37a 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7c24ed4f45 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/testReport/ |
   | Max. process+thread count | 3858 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629801332



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -302,14 +343,28 @@ protected Cell parseCell() throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
       pos += elemLen;
 
-      // timestamp, type and value
-      int tsTypeValLen = length - pos;
+      // timestamp
+      long ts = StreamUtils.readLong(in);
+      pos = Bytes.putLong(backingArray, pos, ts);
+      // type and value
+      int typeValLen = length - pos;
       if (tagsLength > 0) {
-        tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+        typeValLen = typeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+      }
+      // high bit of type byte is 1 if value is compressed
+      byte type = (byte)in.read();
+      if ((type & 0x80) == 0x80) {

Review comment:
       I had a change of heart about this.
   
   We are using a dictionary over the whole file, so your point about amortization is good.
   
   It also gets rid of leakage into KeyValue of the magic high bit if we just assume all values are compressed if value compression is enabled. 




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush at the end of every cell to ensure each cell record persists all of the value data into the expected place. Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to build its dictionary across the whole file's contents, achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629757734



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/region/WAL.proto
##########
@@ -32,6 +32,7 @@ message WALHeader {
   optional bool has_tag_compression = 3;
   optional string writer_cls_name = 4;
   optional string cell_codec_cls_name = 5;
+  optional bool has_value_compression = 6;

Review comment:
       Yes, we initialize the CompressionContext from the header. It has to be done up front. 
   
   If we made value compression something we always do, then this field is not necesary. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629757734



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/region/WAL.proto
##########
@@ -32,6 +32,7 @@ message WALHeader {
   optional bool has_tag_compression = 3;
   optional string writer_cls_name = 4;
   optional string cell_codec_cls_name = 5;
+  optional bool has_value_compression = 6;

Review comment:
       Yes, we initialize the CompressionContext from the header when initializing the reader. It has to be done up front. 
   
   If we made value compression something we always do, then this field is not necessary. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634832170



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/region/WAL.proto
##########
@@ -32,6 +32,8 @@ message WALHeader {
   optional bool has_tag_compression = 3;
   optional string writer_cls_name = 4;
   optional string cell_codec_cls_name = 5;
+  optional bool has_value_compression = 6;
+  optional uint32 value_compression_codec = 7;

Review comment:
       `value_compression_algorithm` it is




-- 
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.

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



[GitHub] [hbase] apurtell merged pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell merged pull request #3244:
URL: https://github.com/apache/hbase/pull/3244


   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634845166



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       You can think of all values in a WAL file where value compression is enabled as chunks of the output of a single compression stream, threaded among the rest of the WAL bits.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r635778623



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/BoundedDelegatingInputStream.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This is a stream that will only supply bytes from its delegate up to a certain limit.
+ * When there is an attempt to set the position beyond that it will signal that the input
+ * is finished.
+ */
+@InterfaceAudience.Private
+public class BoundedDelegatingInputStream extends DelegatingInputStream {

Review comment:
       I thought we could keep them both. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634845166



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       You can think of all values in a WAL file where value compression is enabled as chunks of the output of a single compression stream. The input of this stream is the concatenation of all of the cell value data that is being written into the WAL. The output of this stream is compressed cell value data, stored where uncompressed value data would otherwise be stored. We flush the compression stream at every WALedit so there will never be a short read at read time, when we are processing the thread of compressed bits with the decompressor. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634845166



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       You can think of all values in a WAL file where value compression is enabled as chunks of the output of a single compression stream threaded among the rest of the WAL. The input of this stream is the concatenation of all of the cell value data that is being written into the WAL. The output of this stream is compressed cell value data, stored where uncompressed value data would otherwise be stored. We flush the compression stream at every WALedit so there will never be a short read at read time, when we are processing the thread of compressed bits with the decompressor. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834960180


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   2m  1s |  root in master failed.  |
   | -1 :x: |  compile  |   0m 37s |  hbase-server in master failed.  |
   | -1 :x: |  shadedjars  |   6m  6s |  branch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  8s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 46s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 40s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   6m  7s |  patch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  9s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 48s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 39s |  hbase-server in the patch failed.  |
   |  |   |  27m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2944aeb6f8f8 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 02b018cf1a |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/branch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/branch-shadedjars.txt |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/testReport/ |
   | Max. process+thread count | 340 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840922267


   **Microbrenchmark Results**
   
   Site configuration used:
   
       <!-- retain all WALs  -->
       <property>
         <name>hbase.master.logcleaner.ttl</name>
         <value>604800000</value>
       </property>
       <!-- enable compression -->
       <property>
        <name>hbase.regionserver.wal.enablecompression</name>
        <value>true</value>
       </property>
       <!-- enable value compression -->
       <property>
        <name>hbase.regionserver.wal.value.enablecompression</name>
        <value>true</value>
       </property>
       <!-- set value compression algorithm —>
       <property>
        <name>hbase.regionserver.wal.value.compression.type</name>
        <value>snappy</value>
       </property>
   
   Loader: IntegrationTestLoadCommonCrawl
   
   Input: s3n://commoncrawl/crawl-data/CC-MAIN-2021-10/segments/1614178347293.1/warc/CC-MAIN-20210224165708-20210224195708-00000.warc.gz
   
   Mode | WALs aggregate size | WALs aggregate size difference | WAL writer append time (ms avg)
   -- | -- | -- | --
   Default | 5,117,369,553 | - | 0.290 (stdev 0.328)
   Compression enabled, value compression not enabled | 5,002,683,600 | (2.241%) | 0.372 (stddev 0.336)
   ~~Compression enabled, value compression enabled, v1 patch, Deflate (best speed)~~  | ~~1,209,947,515~~  | ~~(76.4%)~~  | ~~12.694 (stddev 8.48)~~ 
   Compression enabled, value compression enabled, v2 patch, algorithm=SNAPPY | 1,614,336,180 | (68.5%) | 2.971 (stddev 2.029)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best speed) | 1,076,863,466 | (78.95%) | 4.630 (stddev 4.050)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (default) | 1,075,140,058 | (78.99%) | 6.191 (stddev 4.974)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best compression) | 941,505,173 | (81.6%) | 26.322 (stddev 13.38)
   Compression enabled, value compression enabled, v2 patch, algorithm=GZ | 1,082,470,144 | (78.8%) | 27.962 (stddev 15.276)
   
   
   


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837051331


   **WAL Compression Results**
   
   Site configuration:
   
       <!-- retain all WALs  -->
       <property>
          <name>hbase.master.logcleaner.ttl</name>
          <value>604800000</value>
       </property>
   
       <!-- to enable compression -->
       <property>
           <name>hbase.regionserver.wal.enablecompression</name>
           <value>true</value>
       </property>
   
       <!-- to enable value compression -->
       <property>
           <name>hbase.regionserver.wal.value.enablecompression</name>
           <value>true</value>
       </property>
   
   IntegrationTestLoadCommonCrawl
   Input: s3n://commoncrawl/crawl-data/CC-MAIN-2021-10/segments/1614178347293.1/warc/CC-MAIN-20210224165708-20210224195708-00000.warc.gz
   
   | Mode | WALs aggregate size | Difference |
   | -- | -- | -- |
   Default | 5,006,963,824 | - |
   Compression enabled, value compression not enabled | 5,006,874,201 | (0.1%) |
   Compression enabled, value compression enabled | 940,657,251 | (81.2%) |
   
   In this test case WAL compression without value compression is not enough. The schema is already optimized for space efficiency: column families and qualifiers are single characters. There is a bit of redundancy that can be reclaimed but the bulky values (web crawl results) dominate. 


-- 
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.

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629777261



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       The append is single threaded, so I do not think the synchronized itself would cause much overhead.
   
   But for performance, on modern hardwares, with powerful NIC and SSD, now the bottleneck may become CPU so enable compression itself may impact performance.
   
   But anyway, you could choose not enable it if the bottleneck on your cluster is CPU so it is not a blocker for adding compression support.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629759791



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +278,26 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      byte[] buffer = new byte[4096];
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      boolean finished = false;
+      do {
+        int bytesOut = deflater.deflate(buffer);
+        if (bytesOut > 0) {
+          baos.write(buffer, 0, bytesOut);
+        } else {
+          bytesOut = deflater.deflate(buffer, 0, buffer.length, Deflater.SYNC_FLUSH);

Review comment:
       Yes you are missing the semantics of Deflator#deflate. 
   
   Deflator#deflate will only return 0 if it needs more input. At this point because there is no more input, we are now done, but we still need to flush. To flush it we call deflate() again using the method that allows us to specify a sync flag. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is cell by cell. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. It is not necessary to do that. By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of... it builds its dictionary across the whole file. Even though compression is flushed at every cell, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630662113



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -349,6 +424,42 @@ private static void checkLength(int len, int max) throws IOException {
         throw new IOException("Invalid length for compresesed portion of keyvalue: " + len);
       }
     }
+
+    private void readCompressedValue(InputStream in, byte[] outArray, int outOffset,
+        int expectedLength) throws IOException {
+      // Read the size of the compressed value. We serialized it as a vint32.
+      int compressedLength = StreamUtils.readRawVarint32(in);
+      // Read all of the compressed value into a buffer for the Inflater.
+      byte[] buffer = new byte[compressedLength];
+      IOUtils.readFully(in, buffer, 0, compressedLength);
+      // Inflate the compressed value. We know the uncompressed size. Inflator#inflate will
+      // return nonzero for as long as some compressed input remains, and 0 when done.
+      Inflater inflater = compression.getValueCompressor().getInflater();

Review comment:
       Here I think it is better to use the buffer based API instead of a stream, because we are inflating into place into the backing array for cell data. 




-- 
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.

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



[GitHub] [hbase] ndimiduk commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845378079


   I have what's available.


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629801863



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +246,27 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();
+      if (compression.getValueCompressor() != null &&
+          cell.getValueLength() > VALUE_COMPRESS_THRESHOLD) {
+        // Try compressing the cell's value
+        byte[] compressedBytes = compressValue(cell);
+        // Only write the compressed value if we have achieved some space savings.
+        if (compressedBytes.length < cell.getValueLength()) {
+          // Set the high bit of type to indicate the value is compressed
+          out.write((byte)(type|0x80));

Review comment:
       It is simpler if we don't try to conditionally compress values. No high bit twiddling. So there is no leakage into KeyValue in this case and no magic values. 
   
   There also doesn't need to be a size threshold. Since we are using a single deflator instance over all values in the WAL file, overheads of too small values should be amortized. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-835547190


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 30s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 54s |  master passed  |
   | +1 :green_heart: |  shadedjars  |  10m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 20s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 25s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  2s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  5s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 141m 13s |  hbase-server in the patch passed.  |
   |  |   | 183m 15s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 95308bf6c357 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8c2332d465 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/3/testReport/ |
   | Max. process+thread count | 3982 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839070312


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 35s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 44s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 45s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 31s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 31s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 46s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   1m 43s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  22m 13s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   8m 15s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 38s |  The patch does not generate ASF License warnings.  |
   |  |   |  74m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux c1adcec0a2e0 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | Max. process+thread count | 87 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634843684



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       I added a comment in the source.
   
         // We handle input as a sequence of byte[] arrays (call them segments), with
         // DelegatingInputStream providing a way to switch in a new segment when the old
         // segment has been fully consumed.
         // Originally I looked at using BoundedInputStream but you can't reuse/reset the
         // BIS instance, and we can't just create new streams each time around because
         // that would reset compression codec state, which must accumulate over all values
         // in the file in order to build the dictionary in the same way as the compressor
         // did.
   




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629763436



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       There is no sharing. One CompressionContext instance per WAL writer. 
   
   If there is sharing, WAL compression would already be thoroughly broken, that is bad bad bad, dictionaries would be all messed up because they would be updated with output spread among different files and not the same file (think about it, it cannot work) 




-- 
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.

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



[GitHub] [hbase] shahrs87 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
shahrs87 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629708006



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +278,26 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      byte[] buffer = new byte[4096];
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      boolean finished = false;
+      do {
+        int bytesOut = deflater.deflate(buffer);
+        if (bytesOut > 0) {
+          baos.write(buffer, 0, bytesOut);
+        } else {
+          bytesOut = deflater.deflate(buffer, 0, buffer.length, Deflater.SYNC_FLUSH);

Review comment:
       If we reach this else condition that means we have compressed all the data and written to ByteArrayOutputStream. 
   Now in else condition, we are sure that bytesOut will be 0. We are not resetting `buffer`  anywhere. So for the last chunk of Cell's value whose size <= 4096, will we be compressing them twice ?
   @apurtell  It is entirely possible that I misunderstood something. Please correct me if that is the case. Thank you !




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-838765223


   There was a period of time where there was a bad patch. I pushed a replacement. The latest precommits look better. There is one test to fix, back soon. The reason it fails now is after the latest changes it is now an invalid test.


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634841914



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       This is done so we handle input as a sequence of byte[] arrays, with ByteArrayInputStream providing the appropriate signals to the upper decompression stream as to when input is fully consumed. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834966081


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  6s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   1m 59s |  root in master failed.  |
   | -1 :x: |  compile  |   0m 44s |  hbase-server in master failed.  |
   | +1 :green_heart: |  checkstyle  |   1m 35s |  master passed  |
   | -1 :x: |  spotbugs  |   0m 38s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 47s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  cc  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 40s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  checkstyle  |   1m 34s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  hadoopcheck  |   2m  1s |  The patch causes 16 errors with Hadoop v3.1.2.  |
   | -1 :x: |  hadoopcheck  |   4m  4s |  The patch causes 16 errors with Hadoop v3.2.1.  |
   | -1 :x: |  hadoopcheck  |   6m 10s |  The patch causes 16 errors with Hadoop v3.3.0.  |
   | -1 :x: |  hbaseprotoc  |   0m 37s |  hbase-server in the patch failed.  |
   | -1 :x: |  spotbugs  |   0m 31s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 31s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m 35s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 38e3e840b8a9 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 02b018cf1a |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/branch-compile-hbase-server.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/branch-spotbugs-hbase-server.txt |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | cc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-javac-3.1.2.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-javac-3.2.1.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-javac-3.3.0.txt |
   | hbaseprotoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-hbaseprotoc-hbase-server.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-general-check/output/patch-spotbugs-hbase-server.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837776421


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 17s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 16s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 15s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 15s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 46s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 51s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  | 150m 10s |  hbase-server in the patch failed.  |
   |  |   | 187m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 5e05eb7fd3fa 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2b6a91a1da |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/testReport/ |
   | Max. process+thread count | 4782 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844459128


   @bharathv I wrote a simple bounded delegating input stream impl to avoid the unnecessary copy at decompression time. Rebased on master. 
   
   Let me collect updated microbenchmarks for SNAPPY with the latest patch and come back here with them soon.


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837847210


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  4s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 39s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 25s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 37s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 23s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 23s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 47s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  18m  5s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 59s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  2s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 36s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux af1156982b7d 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2b6a91a1da |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r628662332



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -220,6 +223,8 @@ public ByteString compress(byte[] data, Enum dictIndex) {
     }
   }
 
+  static final int VALUE_COMPRESS_THRESHOLD = 100;

Review comment:
       This is arbitrary and will be revisited. Will do some research on the minimum useful plaintext size for Deflate and some testing with various types of data. 
   
   I don't think it makes sense to make this configurable. Virtually no operator will understand how to do better than what we choose now. 




-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r635463483



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       I think @Apache9 is right, just extend FIS and add setDelegate (that makes it small).




-- 
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.

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



[GitHub] [hbase] bharathv commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-841467863


   Thanks for the detailed perf results, SNAPPY seems like the sweet spot. Avg append time took a good hit? (10x IIUC?), something to be concerned about?
   
   Btw, is this ready for review or are you still iterating on the code?


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844355147


   >  just extend FIS and add setDelegate (that makes it small).
   
   Oh, I misunderstood. Sure, I can do that. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r628956456



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -220,6 +223,8 @@ public ByteString compress(byte[] data, Enum dictIndex) {
     }
   }
 
+  static final int VALUE_COMPRESS_THRESHOLD = 100;

Review comment:
       As an arbitrary threshold this isn't a terrible choice. Did some research. [Antirez's smaz library readme](https://github.com/antirez/smaz) has this comment: _To compare this with other libraries, think that like zlib will usually not be able to compress text shorter than 100 bytes._ It's just one data point but someone who put thought into this space came to that conclusion, it's acceptable as a starting point. 
   
   Based on my experimentation, zlib will compress a buffer of 100 bytes filled with zeros to 14 bytes. If we go by powers of 10, 100 seems fine, 10 seems not. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837823204


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 15s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 15s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 11s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 47s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 47s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  |   8m  3s |  hbase-server in the patch failed.  |
   |  |   |  44m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 79e49e6c0241 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2b6a91a1da |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/testReport/ |
   | Max. process+thread count | 806 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836091985


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 19s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 42s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 21s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 21s |  the patch passed  |
   | +1 :green_heart: |  javac  |   5m 21s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 39s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  18m  1s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  1s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 735dda148162 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8c2332d465 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/5/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634836852



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
##########
@@ -151,6 +159,16 @@ public void seek(long pos) throws IOException {
    */
   protected abstract boolean hasTagCompression();
 
+  /**
+   * @return Whether value compression is enabled for this log.
+   */
+  protected abstract boolean hasValueCompression();
+
+  /**
+   * @return Value compression algorithm for this log.
+   */
+  protected abstract Compression.Algorithm getValueCompressionType();

Review comment:
       Yes




-- 
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.

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634849379



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       It is inside JDK.
   
   java.io.FilterInputStream




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844683025


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 55s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 14s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 38s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 14s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  6s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 142m 29s |  hbase-server in the patch passed.  |
   |  |   | 185m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b02b5d6634e6 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe47557f4c |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/testReport/ |
   | Max. process+thread count | 3872 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837831402


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m  0s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 37s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 59s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  |   8m 48s |  hbase-server in the patch failed.  |
   |  |   |  51m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 03fb346176f4 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2b6a91a1da |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/testReport/ |
   | Max. process+thread count | 887 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/7/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834987575


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   1m 47s |  root in master failed.  |
   | -1 :x: |  compile  |   0m 39s |  hbase-server in master failed.  |
   | -1 :x: |  shadedjars  |   6m  3s |  branch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  6s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 49s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 38s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 38s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   6m 10s |  patch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  6s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 44s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 45s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 38s |  hbase-server in the patch failed.  |
   |  |   |  27m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux f8097813ecc4 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 02b018cf1a |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/branch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/branch-shadedjars.txt |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/testReport/ |
   | Max. process+thread count | 339 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629763436



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       There is no sharing. One CompressionContext instance per WAL writer. 
   
   If there is sharing, WAL compression would already be thoroughly broken, that is bad bad bad, dictionaries would be all messed up because they would be updated with output going to different files (think about it, it cannot work) 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-843707286


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  1s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 23s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 21s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 21s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 49s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   0m 11s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  The patch passed checkstyle in hbase-common  |
   | +1 :green_heart: |  checkstyle  |   1m  1s |  hbase-server: The patch generated 0 new + 26 unchanged - 2 fixed = 26 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  17m 50s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  1s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  0s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 38s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 7d6a524aeb00 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 741b4b4674 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634845166



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       You can think of all values in a WAL file where value compression is enabled as chunks of the output of a single compression stream. The input of this stream is the concatenation of all of the values in the WAL. The output of this stream is stored where uncompressed value data would otherwise be stored. We flush the compression stream at every WALedit so there will never be a short read. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839506638


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 39s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 14s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 49s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m  9s |  hbase-server in the patch passed.  |
   |  |   | 184m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux bfe52830216f 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/testReport/ |
   | Max. process+thread count | 4999 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844561641


   I am redoing microbenchmarks with the latest patch and will update here soon. Improvements have unlocked IO performance improvement from the compression. 
   
   |Algorithm|Avg WAL append in ms - Before|Avg WAL append in ms - Now|
   |--|--|--|
   |None|0.290 (stdev 0.328)|unchanged|
   |Compression, but no value compression|0.372 (stddev 0.336)|unchanged|
   |Compression, value compression=SNAPPY|2.971 (stddev 2.029)|**0.0272 (stddev 0.204)**|
   
   Before, SNAPPY+copy had a 10x loss, now SNAPPY+no-copy has a 10x gain. I double checked these findings and it seems correct. I will re-run the benchmark when measuring for the other codec types for comparison. 
   
   Microbenchmarks are collected with [this change](https://gist.github.com/apurtell/596310d08b5ad75cd9677466d36360e4).
   Statistics are collected over the lifetime of the regionserver and are dumped at end of test at shutdown. Statistics are updated under synchronization but this is done in a way that excludes that overhead from measurement. The normal patch does not contain either the instrumentation or the synchronization point. Nanoseconds are converted to milliseconds for the table. 
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839407299


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 10s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  6s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 17s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 36s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 36s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 14s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  4s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 143m 40s |  hbase-server in the patch passed.  |
   |  |   | 183m 13s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux ece61faa51ba 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/testReport/ |
   | Max. process+thread count | 3886 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] bharathv edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844625565


   > SNAPPY+copy had a 10x loss, now SNAPPY+no-copy has a 10x gain. I double checked these findings and it seems correct. I will re-run the benchmark when measuring for the other codec types for comparison.
   
   Holy guacamole! Is this because of the reduced disk IO with compressed values? I'm glad we regained all the lost performance by eliding the copy.
   
   Edit: Just saw your last comment "When using SNAPPY or ZSTD we derive a performance benefit due to reduced IO for the large values in the test case." Sweeeeet!


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837512098


   bq. There is a potential performance improvement here. We could create a Deflater/Inflater pair per column family. 
   
   This idea didn't pan out. It makes compression in my test case slightly worse. 
   
   Let me collect some microbenchmarks for the Deflate compression level options of DEFAULT, BEST_SPEED, and BEST_COMPRESSION with the latest code and present them here. Basically, BEST_COMPRESSION is twice as slow on average as DEFAULT for marginal gains, and BEST_COMPRESSION is twice as fast as DEFAULT but does not compress as well, but good enough, and speed counts here, so BEST_SPEED is best. (However I collected these measures with dictionary per cf so need to redo now that I've dropped the idea.)


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837666043


   I have some changes coming soon. Accepted @bharathv 's argument that unconditionally compressing values if value compression is enabled is fine even if some value cases may not compress because gains and losses are amortized over all values in the WAL file. 
   
   Also address several points of @ndimiduk 's feedback by mooting the discussion. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837063440


   There is a potential performance improvement here. We could create a Deflater/Inflater pair per column family. The universe of column families across all schema in a production setting will not be too large. Then in effect we build a dictionary for each column family, accounting for data distribution differences among the families, which is likely to boost compression results. In addition we can opt for BEST_SPEED under these circumstances for less overall performance impact. Let me explore this idea and come back here. 


-- 
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.

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



[GitHub] [hbase] ndimiduk commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r635659322



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.wal;

Review comment:
       I don't know this history of this package, and especially since we now use a region from inside of the master, i don't know which package is better. I strongly prefer we keep related classes confined to a common package and clamp down accessors as much as we can ; he helps maintain hygiene throughout the codebase.
   
   I feel strongly in that it should be done eventually. I don't feel strongly about you addressing this issue in this specific change. I guess your purpose is not to improve the codebase, but rather to implement this feature, a position to which I am sympathetic.




-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r635463483



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       I think @Apache9 is right, just extend FIS and add setDelegate (that makes tit small).

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       > Originally I looked at using BoundedInputStream but you can't reuse/reset the // BIS instance, and we can't just create new streams each time around because // that would reset compression codec state, which must accumulate over all values // in the file in order to build the dictionary in the same way as the compressor // did.
   
   This is exactly what I was thinking, but haven't thought about the underlying compression dictionary state. I think we can get to this optimization in some other patch if needed. (nice comment btw)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));
+        compressedIn = algorithm.createDecompressionStream(lowerIn, algorithm.getDecompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerIn.setDelegate(new ByteArrayInputStream(inBuffer));
+      }
+      return compressedIn.read(outArray, outOffset, outLength);

Review comment:
       Ack




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629757734



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/region/WAL.proto
##########
@@ -32,6 +32,7 @@ message WALHeader {
   optional bool has_tag_compression = 3;
   optional string writer_cls_name = 4;
   optional string cell_codec_cls_name = 5;
+  optional bool has_value_compression = 6;

Review comment:
       Yes, we initialize the `CompressionContext` from the header when initializing the reader. With this feature toggle placed in `CompressionContext` this is what we have to do. 
   
   If we made value compression something we always do, then this field is not necessary. I'd prefer that over turning it on on-demand at the first encounter of a high bit set in `Type`. 

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -230,6 +230,11 @@ public static long getKeyDataStructureSize(int rlength, int flength, int qlength
     DeleteColumn((byte)12),
     DeleteFamily((byte)14),
 
+    // Effective maximum is 127 (Byte.MAX_VALUE). We set the high order bit of the
+    // type byte in the WAL codecs to indicate, in a backwards compatible way, if the
+    // value is compressed there.
+    EffectiveMaximum((byte)Byte.MAX_VALUE),
+
     // Maximum is used when searching; you look from maximum on down.
     Maximum((byte)255);

Review comment:
       The short answer is this part is where I thought people would raise objections, so wasn't sure what to do here, but I agree with your point. We can try it. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629761476



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/region/WAL.proto
##########
@@ -32,6 +32,7 @@ message WALHeader {
   optional bool has_tag_compression = 3;
   optional string writer_cls_name = 4;
   optional string cell_codec_cls_name = 5;
+  optional bool has_value_compression = 6;

Review comment:
       > I'm surprised to see that we already have WAL compression, but don't explicitly track it in the header.
   
   Huh, I just noticed this too. Something to correct as a followup. 




-- 
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.

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



[GitHub] [hbase] shahrs87 edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
shahrs87 edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-838959572






-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-841585883


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  7s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 48s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m 13s |  hbase-server in the patch passed.  |
   |  |   | 184m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d95758341192 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8ae4d65aa5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/testReport/ |
   | Max. process+thread count | 4096 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837688241


   Rolled up the first round of review feedback into d4d707d 


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837051331


   **WAL Compression Results**
   
   Site configuration:
   
       <!-- retain all WALs  -->
       <property>
          <name>hbase.master.logcleaner.ttl</name>
          <value>604800000</value>
       </property>
   
       <!-- to enable compression -->
       <property>
           <name>hbase.regionserver.wal.enablecompression</name>
           <value>true</value>
       </property>
   
       <!-- to enable value compression -->
       <property>
           <name>hbase.regionserver.wal.value.enablecompression</name>
           <value>true</value>
       </property>
   
   IntegrationTestLoadCommonCrawl
   Input: s3n://commoncrawl/crawl-data/CC-MAIN-2021-10/segments/1614178347293.1/warc/CC-MAIN-20210224165708-20210224195708-00000.warc.gz
   
   | Mode | WALs aggregate size | Difference |
   | -- | -- | -- |
   Default | 5,006,963,824 | - |
   Compression enabled, value compression not enabled | 5,006,874,201 | (0.1%) |
   Compression enabled, value compression enabled | 940,657,251 | (81.2%) |
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-835501037


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  6s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 28s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 40s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 32s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 26s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 26s |  the patch passed  |
   | +1 :green_heart: |  javac  |   5m 26s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  18m  6s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  3s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  6s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 51s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux d0077d9c238e 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8c2332d465 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/3/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush the compressor at the end of every value to ensure each WALedit record persists all of the value data into the expected place. Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to build its dictionary across the contents of all of the values in the file, not just each value considered in isolation (that was the original patch but I pushed an improvement that aligns with this suggestion later), achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629801332



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -302,14 +343,28 @@ protected Cell parseCell() throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
       pos += elemLen;
 
-      // timestamp, type and value
-      int tsTypeValLen = length - pos;
+      // timestamp
+      long ts = StreamUtils.readLong(in);
+      pos = Bytes.putLong(backingArray, pos, ts);
+      // type and value
+      int typeValLen = length - pos;
       if (tagsLength > 0) {
-        tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+        typeValLen = typeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+      }
+      // high bit of type byte is 1 if value is compressed
+      byte type = (byte)in.read();
+      if ((type & 0x80) == 0x80) {

Review comment:
       I had a change of heart about this.
   
   We are using a dictionary over all of the values in the whole WAL, so your point about amortization is good.
   
   It also gets rid of leakage into KeyValue of the magic high bit if we just assume all values are compressed if value compression is enabled. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836051555


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 14s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 16s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 10s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 46s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 47s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m 20s |  hbase-server in the patch passed.  |
   |  |   | 185m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 10a910d8547a 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8c2332d465 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/4/testReport/ |
   | Max. process+thread count | 4133 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630326851



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       You are sizing your buffer too small. Read the Deflater/Inflater javadoc and code to learn more, please. Do let me know if I am doing something wrong. As far as I am aware, my API usage is correct. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629763436



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       There is no sharing. One Compressor instance per WAL writer. 




-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629760196



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       All the inflate deflate methods are synchronized on a single object, for example,  Is that a a concern for bottleneck since we are sharing instances of them, just curious.
   
   ```
   >   public int inflate(byte[] b, int off, int len)
   >         throws DataFormatException
   >     {
   >         if (b == null) {
   >             throw new NullPointerException();
   >         }
   >         if (off < 0 || len < 0 || off > b.length - len) {
   >             throw new ArrayIndexOutOfBoundsException();
   >         }
   >         synchronized (zsRef) {  <===
    >             ensureOpen();
   >             int thisLen = this.len;
   >             int n = inflateBytes(zsRef.address(), b, off, len);
   >             bytesWritten += n;
   >             bytesRead += (thisLen - this.len);
   >             return n;
   >         }
   >     }
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -302,14 +343,28 @@ protected Cell parseCell() throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
       pos += elemLen;
 
-      // timestamp, type and value
-      int tsTypeValLen = length - pos;
+      // timestamp
+      long ts = StreamUtils.readLong(in);
+      pos = Bytes.putLong(backingArray, pos, ts);
+      // type and value
+      int typeValLen = length - pos;
       if (tagsLength > 0) {
-        tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+        typeValLen = typeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+      }
+      // high bit of type byte is 1 if value is compressed
+      byte type = (byte)in.read();
+      if ((type & 0x80) == 0x80) {

Review comment:
       
   ....... My original comment .....
   Why this and why not,
   
   ```
   if (header.has_value_compression) {
    ... readCompressed
   }  else {
    ...
   }
   ```
   
   Value compression is a property of WALCellCodec which is created as a part of WAL reader/writer at init. Why do we have to include this compression bit as a part of every Cell type if we already have it as a part of the header (which I see you added in this patch). So rather than doing this per cell, you could just do it once at code init if has_value_compression == true? Is it possible that some KVs in the same WAL are compressed and some aren't?
   
   ...........
   
   Above was my original comment before reading the full patch, now I see the code about VALUE_COMPRESS_THRESHOLD. I believe this is for for handling the small cell values, right?
   
   Read somewhere that zlib has 18bytes overhead (header trailer and such, don't quote me on this, need to read the original RFC), so even with a million entries, the size increase is not much. Also that will be amortized by larger sized values anyway. On the flip side it makes the code ugly with special type bytes. Isn't it better to just zlib compress all the values? WDYT.
   




-- 
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.

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



[GitHub] [hbase] ndimiduk commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845514956


   > @ndimiduk
   > 
   > > That failure looks suspicious. Let me grab the logs before you merge/close the PR.
   > 
   > How can it be related? That test neither enables WAL compression nor WAL value compression.
   
   Parton Andrew. I did not mean for "suspicious" to imply "caused by this changeset," merely that I was interested in looking at it further. All the PR build artifacts are purged from Jenkins after the PR is closed.


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634841914



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       This is done so we handle input as a sequence of byte[] arrays (call them _segments_), with ByteArrayInputStream providing the appropriate signals to the upper decompression stream as to when input in the current segment is fully consumed. 




-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r635699848



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/BoundedDelegatingInputStream.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This is a stream that will only supply bytes from its delegate up to a certain limit.
+ * When there is an attempt to set the position beyond that it will signal that the input
+ * is finished.
+ */
+@InterfaceAudience.Private
+public class BoundedDelegatingInputStream extends DelegatingInputStream {

Review comment:
       We can merge this and DelegatingInputStream? Don't think we need both of them separately.
   
   ```
   public class BoundedDelegatingInputStream extends FilterInputStream {
    ....   < bounding methods>...
    setDelegate() {}
   }
   ```




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844459128


   @bharathv I wrote a simple bounded delegating input stream impl to avoid the unnecessary copy at decompression time. Rebased on master. 


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845463588


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 35s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 16s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 29s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 34s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 22s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 22s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 48s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   0m 11s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  The patch passed checkstyle in hbase-common  |
   | +1 :green_heart: |  checkstyle  |   1m  2s |  hbase-server: The patch generated 0 new + 26 unchanged - 2 fixed = 26 total (was 28)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  18m 25s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  2s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  4s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   |  66m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 89c6f8671b31 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7c24ed4f45 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839193926


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 48s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m 37s |  hbase-server in the patch passed.  |
   |  |   | 184m 31s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux df8366263ce9 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/testReport/ |
   | Max. process+thread count | 4337 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630293441



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       @shahrs87 https://bugs.java.com/bugdatabase/view_bug.do?bug_id=JDK-8133170 , seems like a known issue with small buffers.




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837512098


   > There is a potential performance improvement here. We could create a Deflater/Inflater pair per column family. 
   
   This idea didn't pan out. It makes compression in my test case slightly worse. 
   
   Let me collect some microbenchmarks for the Deflate compression level options of DEFAULT, BEST_SPEED, and BEST_COMPRESSION with the latest code and present them here. Basically, BEST_COMPRESSION is twice as slow on average as DEFAULT for marginal gains, and BEST_COMPRESSION is twice as fast as DEFAULT but does not compress as well, but good enough, and speed counts here, so BEST_SPEED is best. (However I collected these measures with dictionary per cf so need to redo now that I've dropped the idea.)


-- 
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.

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



[GitHub] [hbase] ndimiduk commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629712475



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -230,6 +230,11 @@ public static long getKeyDataStructureSize(int rlength, int flength, int qlength
     DeleteColumn((byte)12),
     DeleteFamily((byte)14),
 
+    // Effective maximum is 127 (Byte.MAX_VALUE). We set the high order bit of the
+    // type byte in the WAL codecs to indicate, in a backwards compatible way, if the
+    // value is compressed there.
+    EffectiveMaximum((byte)Byte.MAX_VALUE),
+
     // Maximum is used when searching; you look from maximum on down.
     Maximum((byte)255);

Review comment:
       Since this compression bit mask effectively cuts the `Type` value space in half, why introduce a new `EffectiveMaximum`? It seems to me that you can instead replace the value of `Maximum` with the new reduced value of `127`.

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/region/WAL.proto
##########
@@ -32,6 +32,7 @@ message WALHeader {
   optional bool has_tag_compression = 3;
   optional string writer_cls_name = 4;
   optional string cell_codec_cls_name = 5;
+  optional bool has_value_compression = 6;

Review comment:
       I'm surprised to see that we already have WAL compression, but don't explicitly track it in the header.
   
   Why is it needed to add a flag to the header at all? Is this so that a decompressor can be initialized ahead of reading the first compressed `Type` code? Is it better to extend the header than to initialize the decompressor dynamically? Is it reasonable to assume that compression will be enabled by default, and so the decompressor should be created by default and is simply not used if no compressed values are read? (I assume that the `Type` code is written without compression, and thus a decompressor is not required in order to read it back.)

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -230,6 +230,11 @@ public static long getKeyDataStructureSize(int rlength, int flength, int qlength
     DeleteColumn((byte)12),
     DeleteFamily((byte)14),
 
+    // Effective maximum is 127 (Byte.MAX_VALUE). We set the high order bit of the
+    // type byte in the WAL codecs to indicate, in a backwards compatible way, if the
+    // value is compressed there.
+    EffectiveMaximum((byte)Byte.MAX_VALUE),

Review comment:
       I don't find this approach ugly as such. You're using the highest bit as a is-compressed flag, and otherwise retaining the meaning of all existing codes. If I understand you correctly, an uncompressed `Put` has a `Type.code` of `0b0000 0100` while a compressed `Put` has a `Type.code` of `0b1000 0100`. This is fine, so long as we're okay with cutting the size of the `Type` value space in half.
   
   For what it's worth, it might seem like less of a "ugly trick" and more of an "intentional, explicit design" if you were to instead implement as one of the following.
   * Add explicit `Type` entries for `CompressedPut`, `CompressedDelete`, &c., with their associated `code` values also explicitly listed.
   * Fold the representation of compression into the `Type` implementation itself by having each `Type` carry two `code` values (one for compressed, one for uncompressed).
   
   These are simply suggestions; each carry implementation ramifications that I cannot estimate without significantly more code reading.
   
   ---
   
   Returning to this comment after reading the bit where the `Type` code is written. You might consider adding to this Enum an instance method like `int appendTo(OutputStream out)` that encapsulates the details of the `code` encoding.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {

Review comment:
       nit: it's nice for all classes to have a javadoc header.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       nit: why make these symbol public? Can making them public be avoided?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +246,27 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();
+      if (compression.getValueCompressor() != null &&
+          cell.getValueLength() > VALUE_COMPRESS_THRESHOLD) {
+        // Try compressing the cell's value
+        byte[] compressedBytes = compressValue(cell);
+        // Only write the compressed value if we have achieved some space savings.
+        if (compressedBytes.length < cell.getValueLength()) {
+          // Set the high bit of type to indicate the value is compressed
+          out.write((byte)(type|0x80));

Review comment:
       Please keep this value `0x80` as a constant in a named variable, that can be documented back to the `KeyValue$Type` enum, or, better yet, encapsulate it into that object.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -220,6 +223,8 @@ public ByteString compress(byte[] data, Enum dictIndex) {
     }
   }
 
+  static final int VALUE_COMPRESS_THRESHOLD = 100;

Review comment:
       In researching HBASE-25770, I found similar analysis performed by Jetty devs re: gzip compression. I do not know the relationship between gzip and Deflate, but if it's useful, IIRC, Jetty settled on a size threshold of 23 bytes.




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. It is not necessary to do that. By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of... it builds its dictionary across the whole file. Even though compression is flushed at every cell, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844510331


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   6m 27s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 37s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 46s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 54s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 33s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 33s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 45s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   0m  9s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | -0 :warning: |  checkstyle  |   0m 25s |  hbase-common: The patch generated 1 new + 115 unchanged - 0 fixed = 116 total (was 115)  |
   | +1 :green_heart: |  checkstyle  |   1m 10s |  hbase-server: The patch generated 0 new + 26 unchanged - 2 fixed = 26 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 2 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  19m 59s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  3s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 32s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 31s |  The patch does not generate ASF License warnings.  |
   |  |   |  75m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux f89c074ca0e6 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe47557f4c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/artifact/yetus-general-check/output/diff-checkstyle-hbase-common.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844690212


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 47s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 48s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  | 159m 26s |  hbase-server in the patch failed.  |
   |  |   | 196m  8s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 33ea6bbd4f8c 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe47557f4c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/testReport/ |
   | Max. process+thread count | 3627 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840250363


   I took another bite at the @bharathv suggestion to use a streams API instead of driving Deflater and Inflater at the low level. A really excellent additional benefit, if this can be made to work, is we can switch to the Hadoop compression library, which would allow the user to choose among any of our supported algorithms in `Compression` to pick their desired point along the speed versus space continuum. (_The Hadoop compression library kind-of sort-of exports a Deflater/Inflater inspired low level API but it has runtime limitations in this mode requiring native support, which we lack in dev environments and for unit tests. If we use the Hadoop compression library's streaming API, there is no issue._)
   
   I have a change that passes unit tests but is pending a full integration test as well as microbenchmarking. Hopefully I will have all of that done some time tomorrow. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r628956456



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -220,6 +223,8 @@ public ByteString compress(byte[] data, Enum dictIndex) {
     }
   }
 
+  static final int VALUE_COMPRESS_THRESHOLD = 100;

Review comment:
       As an arbitrary threshold this isn't a terrible choice. Did some research. [Antirez's smaz library readme](https://github.com/antirez/smaz) has this comment: _To compare this with other libraries, think that like zlib will usually not be able to compress text shorter than 100 bytes._ It's just one data point but someone who put thought into this space came to that conclusion, it's acceptable as a starting point. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845417490


   @ndimiduk 
   
   > That failure looks suspicious. Let me grab the logs before you merge/close the PR.
   
   How can it be related? That test neither enables WAL compression nor WAL value compression. 
   
       org.junit.runners.model.TestTimedOutException: test timed out after 780 seconds
           at org.apache.hadoop.hbase.master.TestMergeTableRegionsWhileRSCrash.test(TestMergeTableRegionsWhileRSCrash.java:112)
   
   Test log is full of:
   
       java.net.ConnectException: Call to address=33ea6bbd4f8c:42363 null failed on connection exception:
       org.apache.hbase.thirdparty.io.netty.channel.AbstractChannel$AnnotatedConnectException:
       finishConnect(..) failed: Connection refused: 33ea6bbd4f8c/172.17.0.2:42363
   
   Passes on my dev branch:
   
       [INFO] -------------------------------------------------------
       [INFO]  T E S T S
       [INFO] -------------------------------------------------------
       [INFO] Running org.apache.hadoop.hbase.master.TestMergeTableRegionsWhileRSCrash
       [INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 38.5 s - in org.apache.hadoop.hbase.master.TestMergeTableRegionsWhileRSCrash
       [INFO] 
       [INFO] Results:
       [INFO] 
       [INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629759791



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +278,26 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      byte[] buffer = new byte[4096];
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      boolean finished = false;
+      do {
+        int bytesOut = deflater.deflate(buffer);
+        if (bytesOut > 0) {
+          baos.write(buffer, 0, bytesOut);
+        } else {
+          bytesOut = deflater.deflate(buffer, 0, buffer.length, Deflater.SYNC_FLUSH);

Review comment:
       Yes you are missing the semantics of Deflator#deflate. 
   
   Deflator#deflate will only return 0 if it needs more input. At this point because there is no more input, we are now done, but we still need to flush. To flush it we call deflate() again using the method that allows us to specify a sync flag. 
   
   Also, buffer can be reused here like this. It's fine.  buffer does not accumulate output. It is used per pass as part of the API contract with the compressor and then the contents are sent to the output stream. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836053080


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  4s |  https://github.com/apache/hbase/pull/3244 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/5/console |
   | versions | git=2.17.1 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844624849


   SNAPPY or ZSTD are recommended, all other options provided for comparison. (LZMA is included as a sanity check that indeed an expensive algorithm really is expensive.)
   
   When using SNAPPY or ZSTD we derive a performance benefit due to reduced IO for the large values in the test case. 
   
   Microbenchmarks are collected with [this change](https://gist.github.com/apurtell/596310d08b5ad75cd9677466d36360e4).
   Statistics are collected over the lifetime of the regionserver and are dumped at end of test at shutdown. Statistics are updated under synchronization but this is done in a way that excludes that overhead from measurement. The normal patch does not contain either the instrumentation or the synchronization point. Nanoseconds are converted to milliseconds for the table.
   
   Mode | WALs aggregate size | WALs aggregate size difference | WAL writer append time (ms avg)
   -- | -- | -- | --
   Default | 5,117,369,553 | - | 0.290 (stdev 0.328)
   Compression enabled, value compression not enabled | 5,002,683,600 | (2.241%) | 0.372 (stddev 0.336)
   ~~Compression enabled, value compression enabled, v1 patch, Deflate (best speed)~~  | ~~1,209,947,515~~  | ~~(76.4%)~~  | ~~12.694 (stddev 8.48)~~ 
   Compression enabled, value compression enabled, v2 patch, algorithm=SNAPPY | 1,616,387,702 | (68.4%) | 0.027 (stddev 0.204) 
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best speed) | 1,149,008,133 | (77.55%) | 0.043 (stddev 0.195)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (default) | 1,089,241,811 | (78.7%) | 0.056 (stdev 0.310)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best compression) | 941,452,655 | (81.2%) | 0.231 (stddev 1.11)
   _Options below not recommended._ | - | - | -
   Compression enabled, value compression enabled, v2 patch, algorithm=GZ | 1,082,414,015 | (78.9%) | 0.267 (stddev 1.325)
   Compression enabled, value compression enabled, v2 patch, algorithm=LZMA (level 1) | 1,013,951,637 | (80.2%) | 2.157 (stddev 3.302)
   Compression enabled, value compression enabled, v2 patch, algorithm=LZMA (default) | 940,884,618 | (81.7%) | 4.739 (stdev 8.609)


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to builds its dictionary across the whole file's contents, achieving a better compression. We flush at the end of every cell to ensure each cell record persists all of its data. We have to do that anyway... Otherwise the compressor would carry state and bits from previous value over when serializing the next value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.)
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629760897



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -230,6 +230,11 @@ public static long getKeyDataStructureSize(int rlength, int flength, int qlength
     DeleteColumn((byte)12),
     DeleteFamily((byte)14),
 
+    // Effective maximum is 127 (Byte.MAX_VALUE). We set the high order bit of the
+    // type byte in the WAL codecs to indicate, in a backwards compatible way, if the
+    // value is compressed there.
+    EffectiveMaximum((byte)Byte.MAX_VALUE),
+
     // Maximum is used when searching; you look from maximum on down.
     Maximum((byte)255);

Review comment:
       > Add explicit Type entries for CompressedPut, CompressedDelete, &c., with their associated code values also explicitly listed
   
   Such Type entries would not be used anywhere outside of the WAL codec. It's really a WAL codec internal detail. As such, this would raise more questions than provide answers. However I agree with your related point that to make this code production ready the magic constants should be defined with names and javadoc (in wal 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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634841914



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       This is done so we handle input as a sequence of byte[] arrays (call them _segments_), with ByteArrayInputStream providing the appropriate signals to the upper decompression stream as to when input in the current segment is fully consumed. 
   
   Originally I looked at using `BoundedInputStream` but you can't reuse the same instance to feed the compression input stream more input in another segment (because compression stream has a reference to its lower input), and we can't just create both a new IS and a new compression input stream each time around because that would reset compression codec state and fail to decompress the rest of the file. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630326851



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       You are sizing your buffer too small, but I see what you mean. Let me address this in the next update.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634832496



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 

Review comment:
       I use Eclipse not IDEA. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634839182



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();

Review comment:
       We can't set up until we have input, because an IO action is immediately taken. So I do it for both read and write steams to keep the code congruent. 




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844561641


   I am redoing microbenchmarks with the latest patch and will update here soon. Improvements have unlocked IO performance improvement from the compression. 
   
   |Algorithm|Avg WAL append in ms - Before|Avg WAL append in ms - Now|
   |--|--|--|
   |None|0.290 (stdev 0.328)|unchanged|
   |Compression, but no value compression|0.372 (stddev 0.336)|unchanged|
   |Compression, value compression=SNAPPY|2.971 (stddev 2.029)|**0.0272 (stddev 0.204)**|
   
   Before, SNAPPY+copy had a 10x loss, now SNAPPY+no-copy has a 10x gain. I double checked these findings and it seems correct. I will re-run the benchmark when measuring for the other codec types for comparison. 
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839407558


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 47s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m 10s |  hbase-server in the patch passed.  |
   |  |   | 184m  8s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1245e11a2275 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/testReport/ |
   | Max. process+thread count | 4160 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] ndimiduk commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630593612



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       Oh, it is related to your change. Should this new test class, `TestWALSplitValueCompression`, be in the package `org.apache.hadoop.hbase.regionserver.wal` ? I see our WAL implementation and test classes are spread across these two packages.




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-835481646


   I broke trunk on an unrelated issue last night. Rebased. Removed some precommit noise that resulted. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634839182



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();

Review comment:
       So it can be congruent with what we do with output streams, which have to be created at first use not in the constructor.




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839941587


   Latest push addresses findbugs warnings. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634846095



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       Where is `FilterInputStream`?




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush at the end of every cell to ensure each cell record persists all of the value data into the expected place. Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to builds its dictionary across the whole file's contents, achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] Apache9 commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache9 commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836045614


   So we will only compress value?
   
   As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   I think for AsyncFSWAL this is easy as we will buffer all the entries in memory and when flush is called we flush all the in memory data out. For FSHLog it will be a bit hard as we rely on the DFSOutputStream to do flush if the data is too big before we actually call hflush.


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush at the end of every cell to ensure each cell record persists all of the value data into the expected place. We have to do that anyway... Otherwise the compressor would carry state and bits from previous value over when serializing the next value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to builds its dictionary across the whole file's contents, achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-841533295


   > Thanks for the detailed perf results, SNAPPY seems like the sweet spot. Avg append time took a good hit? (10x IIUC?), something to be concerned about?
   
   The extra latency is going to be very much workload dependent, and so WAL value compression should remain a non-default configuration option.
   
   The test case I used to produce this benchmark employs very large values. We are loading in entries from Common Crawl -- web pages, images, javascript, CSS, big xml. So, that is both the source of the good compression results -- the values are large and compressible, generally -- and the source of the extra WAL append latency -- the values are large, generally. Even so, SNAPPY only adding ~2ms per append under these circumstances is a great result. 
   
   For smaller values the latency hit will be proportionally lesser. 
   
   > Btw, is this ready for review or are you still iterating on the code?
   
   See the findbugs warning above and just assume I will fix that. Otherwise I'm waiting for further feedback. 
   
   @bharathv 


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844561641


   I am redoing microbenchmarks with the latest patch and will update here soon. Removing the extra copy unlocked IO performance improvement from the compression. 
   
   |Algorithm|Avg WAL append in ms - Before|Avg WAL append in ms - Now|
   |--|--|--|
   |None|0.290 (stdev 0.328)|unchanged|
   |Compression, but no value compression|0.372 (stddev 0.336)|unchanged|
   |Compression, value compression=SNAPPY|2.971 (stddev 2.029)|**0.0272 (stddev 0.204)**|
   
   Before, SNAPPY+copy had a 10x loss, now SNAPPY+no-copy has a 10x gain. I double checked these findings and it seems correct. I will re-run the benchmark when measuring for the other codec types for comparison. 
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634841914



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       This is done so we handle input as a sequence of byte[] arrays (call them _segments_), with ByteArrayInputStream providing the appropriate signals to the upper decompression stream as to when input in the current segment is fully consumed. 
   
   Originally I looked at using `BoundedInputStream` but you can't reuse the same instance to feed the compression input stream more input in another segment, and we can't just create a new compression input stream because that would reset compression context and fail to decompress the rest of the file. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845559799


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 16s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 15s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m 31s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 14s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 14s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  2s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 24s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 54s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 17s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 225m 19s |  hbase-server in the patch passed.  |
   |  |   | 267m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux bf3385fe68e2 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7c24ed4f45 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/testReport/ |
   | Max. process+thread count | 2571 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/17/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629763436



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       There is no sharing. One CompressionContext instance per WAL writer. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634845166



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       You can think of all values in a WAL file where value compression is enabled as chunks of the output of a single compression stream. The input of this stream is the concatenation of all of the cell value data that is being written into the WAL. The output of this stream is compressed cell value data, stored where uncompressed value data would otherwise be stored. We flush the compression stream at every WALedit so there will never be a short read. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839989889


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 39s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 48s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 47s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 30s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 30s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 45s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   1m 43s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  1s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  20m 31s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  7s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 49s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 33s |  The patch does not generate ASF License warnings.  |
   |  |   |  71m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/11/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 561066370c69 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/11/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/11/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630669258



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize];
+      // Deflater#deflate will return 0 only if more input is required. We iterate until
+      // that condition is met, sending the content of 'buffer' to the output stream at
+      // each step, until deflate returns 0. Then the compressor must be flushed in order
+      // for all of the value's output to be written into the corresponding edit. (Otherwise
+      // the compressor would carry over some of the output for this value into the output
+      // of the next.) To flush the compressor we call deflate again using the method option
+      // that allows us to specify the SYNC_FLUSH flag. The sync output will be placed into
+      // the buffer. When flushing we iterate until there is no more output. Then the flush
+      // is complete and the compressor is ready for more input.
+      int bytesOut;

Review comment:
       I had to back out this change:
   
   Exception in thread "AsyncFSWAL-0-hdfs://localhost:8020/hbase/MasterData" java.lang.AssertionError: should not happen
   	at org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.append(AsyncProtobufLogWriter.java:148)
   	at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.doAppend(AsyncFSWAL.java:773)
   	at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.doAppend(AsyncFSWAL.java:130)
   	at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.appendEntry(AbstractFSWAL.java:1016)
   	at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.appendAndSync(AsyncFSWAL.java:468)
   	at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.consume(AsyncFSWAL.java:556)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: java.io.IOException: write beyond end of stream
   	at java.util.zip.DeflaterOutputStream.write(DeflaterOutputStream.java:201)
   	at org.apache.hadoop.hbase.regionserver.wal.WALCellCodec$CompressedKvEncoder.compressValue(WALCellCodec.java:282)
   




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839367282


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 35s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 28s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 40s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 20s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 20s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 47s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  18m  6s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  3s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  8s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 38s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux e03cba43355d 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/9/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840993130


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 17s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 11s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 11s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 12s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 47s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 49s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m 34s |  hbase-server in the patch passed.  |
   |  |   | 184m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a133cb39ccf3 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / b65733c093 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/testReport/ |
   | Max. process+thread count | 4057 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630659919



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize];
+      // Deflater#deflate will return 0 only if more input is required. We iterate until
+      // that condition is met, sending the content of 'buffer' to the output stream at
+      // each step, until deflate returns 0. Then the compressor must be flushed in order
+      // for all of the value's output to be written into the corresponding edit. (Otherwise
+      // the compressor would carry over some of the output for this value into the output
+      // of the next.) To flush the compressor we call deflate again using the method option
+      // that allows us to specify the SYNC_FLUSH flag. The sync output will be placed into
+      // the buffer. When flushing we iterate until there is no more output. Then the flush
+      // is complete and the compressor is ready for more input.
+      int bytesOut;

Review comment:
       Yes, I think this can work. DeflaterOutputStream has a constructor that will make flush() do SYNC_FLUSH like we need.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630660880



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +245,17 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();
+      out.write(type);
+      if (compression.getValueCompressor() != null) {

Review comment:
       Yes, we can optimize this with a final boolean field. 
   
   Also applies to the check for if tag compression is enabled as well.




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834989576


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   2m 20s |  root in master failed.  |
   | -1 :x: |  compile  |   0m 39s |  hbase-server in master failed.  |
   | -1 :x: |  shadedjars  |   6m  0s |  branch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 15s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   2m  8s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 40s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   6m  9s |  patch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 57s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  4s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 41s |  hbase-server in the patch failed.  |
   |  |   |  29m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1fa7f9218d83 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 02b018cf1a |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/branch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/branch-shadedjars.txt |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/testReport/ |
   | Max. process+thread count | 303 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush the compressor at the end of every value to ensure each WALedit record persists all of the value data into the expected place. Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to build its dictionary across the contents of all of the values in the file, not just each value considered in isolation (that was the original patch but I pushed an improvement that aligns with this suggestion later), achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole block and all of the edits contained within that block, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] ndimiduk commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r633805051



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/region/WAL.proto
##########
@@ -32,6 +32,8 @@ message WALHeader {
   optional bool has_tag_compression = 3;
   optional string writer_cls_name = 4;
   optional string cell_codec_cls_name = 5;
+  optional bool has_value_compression = 6;
+  optional uint32 value_compression_codec = 7;

Review comment:
       nit: call this attribute `value_compression_type` so that it matches the field in `AbstractProtobufLogWriter`, or rename that field to `valueCompressionCodec` so that it matches this attribute. Or call it `value_compression_algorithm` and update the property name down in `ReaderBase`, so that everything aligns with the name of the Java type, which is a `o.a.h.h.io.compress.Compression$Algorithm`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
##########
@@ -165,10 +178,20 @@ public void init(FileSystem fs, Path path, Configuration conf, boolean overwrita
 
     initOutput(fs, path, overwritable, bufferSize, replication, blocksize);
 
-    boolean doTagCompress = doCompress
-        && conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
-    length.set(writeMagicAndWALHeader(ProtobufLogReader.PB_WAL_MAGIC, buildWALHeader(conf,
-      WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))));
+    boolean doTagCompress = doCompress &&

Review comment:
       nit: Duplication of config parsing between `init` and`initializeCompressionContext` is bizarre ; we literally just did this work 8 lines earlier. for future cleanup, I suppose.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,20 +264,31 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private void writeCompressedValue(OutputStream out, Cell cell) throws IOException {
+      byte[] compressed = compression.getValueCompressor().compress(cell.getValueArray(),
+        cell.getValueOffset(), cell.getValueLength());
+      StreamUtils.writeRawVInt32(out, compressed.length);
+      out.write(compressed);
+    }
+
   }
 
   static class CompressedKvDecoder extends BaseDecoder {
     private final CompressionContext compression;
+    private final boolean hasValueCompression;
+    private final boolean hasTagCompression;
     public CompressedKvDecoder(InputStream in, CompressionContext compression) {
       super(in);
       this.compression = compression;
+      this.hasValueCompression = compression.hasValueCompression();
+      this.hasTagCompression = compression.hasTagCompression();
     }
 
     @Override
     protected Cell parseCell() throws IOException {

Review comment:
       Are similar updates needed in `SecureWALCellCodec`? my IDE is telling me that these methods between those two classes have a fair bit of duplicate code.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       So you decided not to move `TestWALSplitValueCompression` into the same package as this class?
   
   It looks like:
   * `ENABLE_WAL_TAGS_COMPRESSION` can be package private as is.
   * `ENABLE_WAL_VALUE_COMPRESSION` can be be package private if `TestWALReplayValueCompression` is moved to `o.a.h.h.regionserver.wal`, and it moves trivially.
   * `WAL_VALUE_COMPRESSION_TYPE` isn't used anywhere else so it can be `private`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
##########
@@ -151,6 +159,16 @@ public void seek(long pos) throws IOException {
    */
   protected abstract boolean hasTagCompression();
 
+  /**
+   * @return Whether value compression is enabled for this log.
+   */
+  protected abstract boolean hasValueCompression();
+
+  /**
+   * @return Value compression algorithm for this log.
+   */
+  protected abstract Compression.Algorithm getValueCompressionType();

Review comment:
       `getValueCompressionAlgorithm` ?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {
+
+  InputStream lowerStream;

Review comment:
       Are you making any claims about using this class in a multi-threaded environment? For instance, should `lowerStream` be marked `volatile` to ensure that multiple consumers of an instance of this class linearize their access to the field? 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 

Review comment:
       delete this newline and you will have completely cleaned up the import checkstyle warnings. Also, if you were unaware, you can have IDEA do this for you automatically when you "Optimize Imports" by using the checkstyle plugin and importing our checkstyle configuration settings. I documented this a while back ; see [our book](http://hbase.apache.org/book.html#_intellij_idea).

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving

Review comment:
       You can omit `Evolving` as it is redundant to `Private` ; `Private` makes no compatibility guarantees version-to-version, and `Evolving` is an explicit marker for breaking compatibility at minor releases.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -70,18 +150,55 @@ public CompressionContext(Class<? extends Dictionary> dictType, boolean recovere
     if (hasTagCompression) {
       tagCompressionContext = new TagCompressionContext(dictType, Short.MAX_VALUE);
     }
+    if (hasValueCompression && valueCompressionType != null) {
+      valueCompressor = new ValueCompressor(valueCompressionType);
+    }
+  }
+
+  public CompressionContext(Class<? extends Dictionary> dictType, boolean recoveredEdits,
+      boolean hasTagCompression)
+      throws SecurityException, NoSuchMethodException, InstantiationException,
+        IllegalAccessException, InvocationTargetException, IOException {
+    this(dictType, recoveredEdits, hasTagCompression, false, null);
+  }
+
+  public boolean hasTagCompression() {
+    return tagCompressionContext != null;
+  }
+
+  public boolean hasValueCompression() {
+    return valueCompressor != null;
   }
 
-  public Dictionary getDictionary(Enum dictIndex) {
+  public Dictionary getDictionary(Enum<DictionaryIndex> dictIndex) {

Review comment:
       Drop `Enum<...>` and make this simply `Dictionary getDictionary(DictionaryIndex dictIndex)`

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {

Review comment:
       `IOException` isn't thrown.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.wal;

Review comment:
       Should be in `o.a.h.h.regionserver.wal` ?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));
+        compressedIn = algorithm.createDecompressionStream(lowerIn, algorithm.getDecompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerIn.setDelegate(new ByteArrayInputStream(inBuffer));
+      }
+      return compressedIn.read(outArray, outOffset, outLength);
+    }
+
+    public void clear() {
+      lowerIn = null;
+      compressedIn = null;
+      lowerOut = null;
+      compressedOut = null;
+    }
+
+  };

Review comment:
       unnecessary `;`




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-841546694


   Pushed a rebase on master and a fix for the latest findbugs warning.


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634828768



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {
+
+  InputStream lowerStream;

Review comment:
       I'm not anticipating multithreaded use of this class but I did put it into common. Let me document this more and use atomic references to future proof. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630657193



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize];
+      // Deflater#deflate will return 0 only if more input is required. We iterate until
+      // that condition is met, sending the content of 'buffer' to the output stream at
+      // each step, until deflate returns 0. Then the compressor must be flushed in order
+      // for all of the value's output to be written into the corresponding edit. (Otherwise
+      // the compressor would carry over some of the output for this value into the output
+      // of the next.) To flush the compressor we call deflate again using the method option
+      // that allows us to specify the SYNC_FLUSH flag. The sync output will be placed into
+      // the buffer. When flushing we iterate until there is no more output. Then the flush
+      // is complete and the compressor is ready for more input.
+      int bytesOut;

Review comment:
       With the latest code this won't let us reuse the buffer needed for Deflator#deflate. Maybe not important. I can look at it.




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-841570261


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 58s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 17s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 40s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 23s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 40s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 22s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 22s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 49s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | -0 :warning: |  checkstyle  |   1m  2s |  hbase-server: The patch generated 1 new + 27 unchanged - 1 fixed = 28 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  18m  1s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  0s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  8s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 22s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux d40b0fb28fc4 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8ae4d65aa5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-843747026


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 14s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 12s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 21s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 17s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 36s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 36s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  7s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  6s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 141m 37s |  hbase-server in the patch passed.  |
   |  |   | 182m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 87f937bcbceb 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 741b4b4674 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/testReport/ |
   | Max. process+thread count | 3797 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/14/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634837188



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,20 +264,31 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private void writeCompressedValue(OutputStream out, Cell cell) throws IOException {
+      byte[] compressed = compression.getValueCompressor().compress(cell.getValueArray(),
+        cell.getValueOffset(), cell.getValueLength());
+      StreamUtils.writeRawVInt32(out, compressed.length);
+      out.write(compressed);
+    }
+
   }
 
   static class CompressedKvDecoder extends BaseDecoder {
     private final CompressionContext compression;
+    private final boolean hasValueCompression;
+    private final boolean hasTagCompression;
     public CompressedKvDecoder(InputStream in, CompressionContext compression) {
       super(in);
       this.compression = compression;
+      this.hasValueCompression = compression.hasValueCompression();
+      this.hasTagCompression = compression.hasTagCompression();
     }
 
     @Override
     protected Cell parseCell() throws IOException {

Review comment:
       Ugh, yes, WTF.
   Ok, will do




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845560449


   Thanks. Unless objection I will merge this tonight or tomorrow morning, to master, and then to branch-2 (for future 2.5.0)


-- 
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.

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



[GitHub] [hbase] Apache9 commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache9 commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837580362


   > > So we will only compress value?
   > 
   > This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too.
   > 
   > > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   > 
   > This is not possible for two reasons:
   > 
   > 1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue.
   > 2. We flush the compressor at the end of every value to ensure each WALedit record persists all of the value data into the expected place. Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to build its dictionary across the contents of all of the values in the file, not just each value considered in isolation (that was the original patch but I pushed an improvement that aligns with this suggestion later), achieving a better compression.
   > 
   > Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole block and all of the edits contained within that block, especially if compression or encryption is enabled.
   
   OK, so the idea is like all the values are written out like a stream, seems workable. Let me take a look at the code then.


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836165849


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 35s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 51s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 16s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 40s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 48s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  5s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 11s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 143m 53s |  hbase-server in the patch passed.  |
   |  |   | 185m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 37d932ab31f2 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8c2332d465 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/5/testReport/ |
   | Max. process+thread count | 4058 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634840283



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {
+
+  InputStream lowerStream;
+
+  public DelegatingInputStream(InputStream lowerStream) {
+    this.lowerStream = lowerStream;
+  }
+
+  public InputStream getDelegate() {
+    return lowerStream;
+  }
+
+  public void setDelegate(InputStream lowerStream) {
+    this.lowerStream = lowerStream;
+  }
+
+  @Override
+  public int read() throws IOException {
+    return lowerStream.read();
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return lowerStream.read(b);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    return lowerStream.read(b, off, len);
+  }
+
+  @Override
+  public long skip(long n) throws IOException {
+    return lowerStream.skip(n);
+  }
+
+  @Override
+  public int available() throws IOException {
+    return lowerStream.available();
+  }
+
+  @Override
+  public void close() throws IOException {
+    lowerStream.close();
+  }
+
+  @Override
+  public synchronized void mark(int readlimit) {
+    lowerStream.mark(readlimit);
+  }
+
+  @Override
+  public synchronized void reset() throws IOException {
+    lowerStream.reset();
+  }
+
+  @Override
+  public boolean markSupported() {

Review comment:
       Did not know about it but I'm good with what I've done here.




-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629779970



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       Ah ya, ring buffer consumer is single threaded, missed that part, right its not a performance concern.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634840135



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));

Review comment:
       We can't set up until we have input, because an IO action is immediately taken. So I do it for both read and write steams to keep the code congruent. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840922267


   **Microbrenchmark Results**
   
   Site configuration used:
   
       <!-- retain all WALs  -->
       <property>
         <name>hbase.master.logcleaner.ttl</name>
         <value>604800000</value>
       </property>
       <!-- enable compression -->
       <property>
        <name>hbase.regionserver.wal.enablecompression</name>
        <value>true</value>
       </property>
       <!-- enable value compression -->
       <property>
        <name>hbase.regionserver.wal.value.enablecompression</name>
        <value>true</value>
       </property>
       <!-- set value compression algorithm —>
       <property>
        <name>hbase.regionserver.wal.value.compression.type</name>
        <value>SNAPPY</value>
       </property>
   
   Loader: IntegrationTestLoadCommonCrawl
   
   Input: s3n://commoncrawl/crawl-data/CC-MAIN-2021-10/segments/1614178347293.1/warc/CC-MAIN-20210224165708-20210224195708-00000.warc.gz
   
   Mode | WALs aggregate size | WALs aggregate size difference | WAL writer append time (ms avg)
   -- | -- | -- | --
   Default | 5,117,369,553 | - | 0.290 (stdev 0.328)
   Compression enabled, value compression not enabled | 5,002,683,600 | (2.241%) | 0.372 (stddev 0.336)
   ~~Compression enabled, value compression enabled, v1 patch, Deflate (best speed)~~  | ~~1,209,947,515~~  | ~~(76.4%)~~  | ~~12.694 (stddev 8.48)~~ 
   Compression enabled, value compression enabled, v2 patch, algorithm=SNAPPY | 1,614,336,180 | (68.5%) | 2.971 (stddev 2.029)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best speed) | 1,076,863,466 | (78.95%) | 4.630 (stddev 4.050)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (default) | 1,075,140,058 | (78.99%) | 6.191 (stddev 4.974)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best compression) | 941,505,173 | (81.6%) | 26.322 (stddev 13.38)
   Compression enabled, value compression enabled, v2 patch, algorithm=GZ | 1,082,470,144 | (78.8%) | 27.962 (stddev 15.276)
   
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634838171



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -70,18 +150,55 @@ public CompressionContext(Class<? extends Dictionary> dictType, boolean recovere
     if (hasTagCompression) {
       tagCompressionContext = new TagCompressionContext(dictType, Short.MAX_VALUE);
     }
+    if (hasValueCompression && valueCompressionType != null) {
+      valueCompressor = new ValueCompressor(valueCompressionType);
+    }
+  }
+
+  public CompressionContext(Class<? extends Dictionary> dictType, boolean recoveredEdits,
+      boolean hasTagCompression)
+      throws SecurityException, NoSuchMethodException, InstantiationException,
+        IllegalAccessException, InvocationTargetException, IOException {
+    this(dictType, recoveredEdits, hasTagCompression, false, null);
+  }
+
+  public boolean hasTagCompression() {
+    return tagCompressionContext != null;
+  }
+
+  public boolean hasValueCompression() {
+    return valueCompressor != null;
   }
 
-  public Dictionary getDictionary(Enum dictIndex) {
+  public Dictionary getDictionary(Enum<DictionaryIndex> dictIndex) {

Review comment:
       I cannot do this without a ripple effect through WALCodec, which is LimitedPrivate. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844301569


   Unit test failure is not related.
   
       TestJMXListener.setupBeforeClass:68 » IO Shutting down
   
   Going to merge today unless objection. 


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840922267


   **Microbrenchmark Results**
   
   Site configuration used:
   
       <!-- retain all WALs  -->
       <property>
         <name>hbase.master.logcleaner.ttl</name>
         <value>604800000</value>
       </property>
       <!-- enable compression -->
       <property>
        <name>hbase.regionserver.wal.enablecompression</name>
        <value>true</value>
       </property>
       <!-- enable value compression -->
       <property>
        <name>hbase.regionserver.wal.value.enablecompression</name>
        <value>true</value>
       </property>
       <!-- set value compression algorithm —>
       <property>
        <name>hbase.regionserver.wal.value.compression.type</name>
        <value>snappy</value>
       </property>
   
   Loader: IntegrationTestLoadCommonCrawl
   
   Input: s3n://commoncrawl/crawl-data/CC-MAIN-2021-10/segments/1614178347293.1/warc/CC-MAIN-20210224165708-20210224195708-00000.warc.gz
   
   SNAPPY or ZSTD at level 1 are recommended, all other options provided for comparison. 
   
   Mode | WALs aggregate size | WALs aggregate size difference | WAL writer append time (ms avg)
   -- | -- | -- | --
   Default | 5,117,369,553 | - | 0.290 (stdev 0.328)
   Compression enabled, value compression not enabled | 5,002,683,600 | (2.241%) | 0.372 (stddev 0.336)
   ~~Compression enabled, value compression enabled, v1 patch, Deflate (best speed)~~  | ~~1,209,947,515~~  | ~~(76.4%)~~  | ~~12.694 (stddev 8.48)~~ 
   Compression enabled, value compression enabled, v2 patch, algorithm=SNAPPY | 1,614,336,180 | (68.5%) | 2.971 (stddev 2.029)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best speed) | 1,076,863,466 | (78.95%) | 4.630 (stddev 4.050)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (default) | 1,075,140,058 | (78.99%) | 6.191 (stddev 4.974)
   Compression enabled, value compression enabled, v2 patch, algorithm=ZSTD (best compression) | 941,505,173 | (81.6%) | 26.322 (stddev 13.38)
   Compression enabled, value compression enabled, v2 patch, algorithm=GZ | 1,082,470,144 | (78.8%) | 27.962 (stddev 15.276)
   
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837773079


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 40s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 12s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 21s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 21s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  5s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  8s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  | 144m  4s |  hbase-server in the patch failed.  |
   |  |   | 184m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b174f5b15e2d 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2b6a91a1da |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/testReport/ |
   | Max. process+thread count | 4145 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] shahrs87 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
shahrs87 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630367777



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       @apurtell  I increased the buffer to 100, changed the input string to 500 and everything works fine. The decompressed string was the same as original raw string.
   On a side note, `Deflater#deflateBytes(long addr, byte[] b, int off, int len, int flush)` which is a native method doesn't have much documentation which help newbies to understand whats going on but not related to this PR.
   Sorry for the noise if I have created. :(




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-838765223


   There was a period of time where there was a bad patch. I pushed a replacement. The latest precommits look better. There is one test to fix, back soon.


-- 
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.

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



[GitHub] [hbase] shahrs87 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
shahrs87 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629711573



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +278,26 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      byte[] buffer = new byte[4096];
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      boolean finished = false;
+      do {
+        int bytesOut = deflater.deflate(buffer);
+        if (bytesOut > 0) {
+          baos.write(buffer, 0, bytesOut);
+        } else {
+          bytesOut = deflater.deflate(buffer, 0, buffer.length, Deflater.SYNC_FLUSH);

Review comment:
       Also the java doc for SYNC_FLUSH says the following thing.
   ```
   In the case of FULL_FLUSH or SYNC_FLUSH, if the return value is len, the space available in output buffer b, this method should be invoked again with the same flush parameter and more output space.
   ```
   https://docs.oracle.com/javase/8/docs/api/java/util/zip/Deflater.html#deflate-byte:A-int-int-int-
   @apurtell  Do we need to change something regarding this comment ?




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837655575


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  2s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 20s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 37s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 27s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 27s |  the patch passed  |
   | +1 :green_heart: |  javac  |   5m 27s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 39s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 50s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  4s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 38s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 185e016f5114 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2b6a91a1da |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/6/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r635482369



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       Sorry, @bharathv helped me see I did not understand your feedback here. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-843686132


   Rebased. 
   
   Addressed last round of review feedback. 
   
   Added another unit test. The split and replay tests which enable value compression already provide similar coverage but there's more going on in those tests than just value compression. New unit just writes a compressed WAL and then reads it back in, making it more clear where the issue may lie, if (value) compression is broken.


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845420407


   Pushed a fix for whitespace and javadoc issues introduced in last change. No additional changes anticipated from this point.


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844641501


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  6s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 43s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 46s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 39s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 36s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 36s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 47s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   0m  8s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | -0 :warning: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 1 new + 115 unchanged - 0 fixed = 116 total (was 115)  |
   | +1 :green_heart: |  checkstyle  |   1m  8s |  hbase-server: The patch generated 0 new + 26 unchanged - 2 fixed = 26 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 2 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  20m 38s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  8s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 44s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 32s |  The patch does not generate ASF License warnings.  |
   |  |   |  71m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux bace79313f46 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe47557f4c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/artifact/yetus-general-check/output/diff-checkstyle-hbase-common.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/16/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837518016


   > Is it reasonable to always use the same compressor all the time for WALs? It strikes me that operators make an explicit choice about the compression used in their tables, so maybe they would expect that the compressor used by WALs would follow suit. I know it's not that simple, given that WALs are per host and not per table, so maybe it's something to consider in the future.
   
   If this is to be tied to table settings we would have to look up settings per table/CF whenever processing an edit. WALs have to be readable on their own without regard to schema changes, which may happen at any point. -1. I don't want to do this. It would be too expensive both in terms of time for all of those lookups and space for each compressed edit to specify algorithm and parameters. 
   
   I see the goal here as making WALs not so terribly un-efficient to keep around. One pass of a standard compression over the values can achieve that, especially when we drop values that don't compress. Speed is important so BEST_SPEED is the right choice for compression level always. Enabling or disabling compression is enough configuration to give to users. This is what we have given them up to now. Now they can turn on or off value compression too. Arguably zlib compression level here is not a user serviceable setting but I could be convinced to at least make that a site configuration setting. 


-- 
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.

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



[GitHub] [hbase] shahrs87 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
shahrs87 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630367777



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       @apurtell  I increased the buffer to 100, changed the input string to 500 and everything works fine. The decompressed string was the same as original raw string.
   On a side note, `Deflater#deflateBytes(long addr, byte[] b, int off, int len, int flush)` which is a native method doesn't have much documentation which help newbies to understand whats going on but not related to this PR.
   




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634852832



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       Found it, but the `FilterInputStream` constructor is not public and there's no API for switching the delegate either here or in implementing classes that are not abstract.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629757222



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +246,27 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();
+      if (compression.getValueCompressor() != null &&
+          cell.getValueLength() > VALUE_COMPRESS_THRESHOLD) {
+        // Try compressing the cell's value
+        byte[] compressedBytes = compressValue(cell);
+        // Only write the compressed value if we have achieved some space savings.
+        if (compressedBytes.length < cell.getValueLength()) {
+          // Set the high bit of type to indicate the value is compressed
+          out.write((byte)(type|0x80));

Review comment:
       > Jetty settled on a size threshold of 23 bytes.
   
   Thank you @ndimiduk . gzip and deflate are the same thing, essentially. Let's opt for the smaller threshold and see how it goes. Worst case if the compressor produces output that is larger than the original, we just discard it and use the original, so that's not a problem. With a smaller threshold more values are eligible for compression so there will be more time spent in compression, but presumably with a pay off in space savings, so that seems fine. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634839349



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+

Review comment:
       It would be good hygiene, yes, will do




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634839182



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();

Review comment:
       I forget exactly which case it was, either input or output, but in one case we can't set up until we have the user supplied stream, because an IO action is immediately taken. So I do it for both read and write steams to keep the code congruent. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634838987



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java
##########
@@ -0,0 +1,44 @@
+/**
+ * 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.wal;

Review comment:
       This is in the same package as `TestWALSplit` and five other similarly named units. Do you want me to move all of them? 




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush at the end of every cell to ensure each cell record persists all of the value data into the expected place. Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to build its dictionary across the contents of all of the values in the file, not just each value considered in isolation (that was the original patch but I pushed an improvement that aligns with this suggestion later), achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629810127



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +246,27 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();
+      if (compression.getValueCompressor() != null &&
+          cell.getValueLength() > VALUE_COMPRESS_THRESHOLD) {
+        // Try compressing the cell's value
+        byte[] compressedBytes = compressValue(cell);
+        // Only write the compressed value if we have achieved some space savings.
+        if (compressedBytes.length < cell.getValueLength()) {
+          // Set the high bit of type to indicate the value is compressed
+          out.write((byte)(type|0x80));

Review comment:
       This was also a hold over from the initial version that had a Inflater/Deflater instance per value, so it would have made a difference then. 




-- 
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.

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



[GitHub] [hbase] bharathv commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844625565


   > SNAPPY+copy had a 10x loss, now SNAPPY+no-copy has a 10x gain. I double checked these findings and it seems correct. I will re-run the benchmark when measuring for the other codec types for comparison.
   
   Holy guacamole! Is this because of the reduced disk IO with compressed values? I'm glad we regained all the lost performance by eliding the copy.


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839244870


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  9s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 54s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 47s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 47s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 16s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  4s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 13s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 205m 32s |  hbase-server in the patch passed.  |
   |  |   | 248m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3efdf6c6cbee 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/testReport/ |
   | Max. process+thread count | 3061 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/8/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634852832



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {

Review comment:
       Found it, but the `FilterInputStream` constructor is not public and there's no API for switching the delegate either there or in implementing classes that are not abstract.




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-838973994


   > Speaking out loud. During RU, the wal written by RS that has ENABLE_WAL_VALUE_COMPRESSION set to false died and another RS which was recently upgraded has the above property set to true. Will the new RS be able to decode the cell ?
   
   @shahrs87  Yes. 
   
   Each WAL file carries metadata about compression particulars for the file, so we can set up the CompressionContext correctly no matter the site file defaults. This logic is in ProtobufLogReader.java. 


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-839431756


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 55s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 21s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 40s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 17s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   7m 12s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 45s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 45s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 48s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | +1 :green_heart: |  checkstyle  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  20m  1s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  4s |  the patch passed  |
   | -1 :x: |  spotbugs  |   2m 24s |  hbase-server generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 33s |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 44s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Unread field:field be static?  At CompressionContext.java:[line 53] |
   |  |  Unread field:field be static?  At CompressionContext.java:[line 54] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 63d74aa98125 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/10/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630410074



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +245,17 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();

Review comment:
       nit: remove local variable assignment

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -349,6 +424,42 @@ private static void checkLength(int len, int max) throws IOException {
         throw new IOException("Invalid length for compresesed portion of keyvalue: " + len);
       }
     }
+
+    private void readCompressedValue(InputStream in, byte[] outArray, int outOffset,
+        int expectedLength) throws IOException {
+      // Read the size of the compressed value. We serialized it as a vint32.
+      int compressedLength = StreamUtils.readRawVarint32(in);
+      // Read all of the compressed value into a buffer for the Inflater.
+      byte[] buffer = new byte[compressedLength];
+      IOUtils.readFully(in, buffer, 0, compressedLength);
+      // Inflate the compressed value. We know the uncompressed size. Inflator#inflate will
+      // return nonzero for as long as some compressed input remains, and 0 when done.
+      Inflater inflater = compression.getValueCompressor().getInflater();

Review comment:
       Same comment as above, use InflaterInputStream?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -302,18 +367,28 @@ protected Cell parseCell() throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
       pos += elemLen;
 
-      // timestamp, type and value
-      int tsTypeValLen = length - pos;
+      // timestamp
+      long ts = StreamUtils.readLong(in);
+      pos = Bytes.putLong(backingArray, pos, ts);
+      // type and value
+      int typeValLen = length - pos;
       if (tagsLength > 0) {
-        tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+        typeValLen = typeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+      }
+      byte type = (byte)in.read();
+      pos = Bytes.putByte(backingArray, pos, type);
+      int valLen = typeValLen - 1;
+      if (compression.hasValueCompression()) {

Review comment:
       Same here on avoiding a branch, should we do that?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {

Review comment:
       q: Is this special handling to avoid an array copy from the bytebuffer? (rather than using Cell interface getValueArray() methods). I see this kinda special handling in multiple places but don't have the context..

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize];
+      // Deflater#deflate will return 0 only if more input is required. We iterate until
+      // that condition is met, sending the content of 'buffer' to the output stream at
+      // each step, until deflate returns 0. Then the compressor must be flushed in order
+      // for all of the value's output to be written into the corresponding edit. (Otherwise
+      // the compressor would carry over some of the output for this value into the output
+      // of the next.) To flush the compressor we call deflate again using the method option
+      // that allows us to specify the SYNC_FLUSH flag. The sync output will be placed into
+      // the buffer. When flushing we iterate until there is no more output. Then the flush
+      // is complete and the compressor is ready for more input.
+      int bytesOut;

Review comment:
       Why not use DeflateInputStream for this? That hides most of the complexity here for us..
   
   ```
        ByteArrayOutputStream bos = new ByteArrayOutputStream(input.length);
        DeflaterOutputStream stream = new DeflaterOutputStream(bos, deflater);
       try {
         stream.write(input);
         stream.close();
       } catch (IOException e) {
          throw 
       }
       return bos.toByteArray();
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +245,17 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();
+      out.write(type);
+      if (compression.getValueCompressor() != null) {

Review comment:
       Should we consider eliminating this branch since this is known at init time and this code path is perf critical?




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630673256



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize];
+      // Deflater#deflate will return 0 only if more input is required. We iterate until
+      // that condition is met, sending the content of 'buffer' to the output stream at
+      // each step, until deflate returns 0. Then the compressor must be flushed in order
+      // for all of the value's output to be written into the corresponding edit. (Otherwise
+      // the compressor would carry over some of the output for this value into the output
+      // of the next.) To flush the compressor we call deflate again using the method option
+      // that allows us to specify the SYNC_FLUSH flag. The sync output will be placed into
+      // the buffer. When flushing we iterate until there is no more output. Then the flush
+      // is complete and the compressor is ready for more input.
+      int bytesOut;

Review comment:
       @bharathv I can come back to try this again, but the first time around code that would seem to work did not. Must have missed something, although it was not obvious. 




-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-836903862


   > So we will only compress value?
   
   This is an enhancement to existing WAL compression. As you know the existing WAL compression already compresses other aspects of WAL entries _except_ for the value. This patch adds support for compressing values too. 
   
   > As we will do batching when writing WAL entries out, is it possible to compress when flushing? The data will be larger and compress may perform better. The structure of a WAL file will be multiple compressed blocks.
   
   This is not possible for two reasons:
   
   1. WALCellCodec does not compress the WAL file in blocks. The design is edit by edit. I want to introduce value compression without re-engineering the whole WAL format. Perhaps our WAL file format is due for a redesign, but I would like to see that be a different issue. 
   
   2. We flush at the end of every value to ensure each WALedit record persists all of the value data into the expected place. Otherwise the compressor would put some of the unflushed output of the previous value into the next/current value. But, we are not resetting the compressor. (That would be FULL_FLUSH. We are using SYNC_FLUSH.) By using the same Deflater instance for the whole WAL we already get the benefit you are thinking of. The (re-used) Deflater is able to build its dictionary across the contents of all of the values in the file, not just each value considered in isolation (that was the original patch but I pushed an improvement that aligns with this suggestion later), achieving a better compression. 
   
   Way back in the distant past our WAL format was based on Hadoop's SequenceFile, which supported both record-by-record and block based compression, where the blocks would contain multiple records. I don't remember why we moved away from it but I imagine it was because if there are corruptions of the WAL, a record by record codec is able to skip over the corrupt record and we lose only the record (or as many records as are actually corrupt), but with a block format we would lose the whole record and all of the edits contained within that record, especially if compression or encryption is enabled. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630657741



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,61 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {

Review comment:
       This didn't work out actually. Yeah, we can just use getValueArray to simplify. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630660880



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +245,17 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();
+      out.write(type);
+      if (compression.getValueCompressor() != null) {

Review comment:
       Yes, we can optimize this with a final boolean field. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844574860


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 53s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  6s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 40s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 46s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 48s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m 59s |  hbase-server in the patch passed.  |
   |  |   | 188m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c76a45c06d4f 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe47557f4c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/testReport/ |
   | Max. process+thread count | 3669 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634841259



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));
+        compressedIn = algorithm.createDecompressionStream(lowerIn, algorithm.getDecompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerIn.setDelegate(new ByteArrayInputStream(inBuffer));
+      }
+      return compressedIn.read(outArray, outOffset, outLength);

Review comment:
       Actual bytes read are returned. It would be up to the caller to handle.
   
   In practice, all bytes are read here. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629802049



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -230,6 +230,11 @@ public static long getKeyDataStructureSize(int rlength, int flength, int qlength
     DeleteColumn((byte)12),
     DeleteFamily((byte)14),
 
+    // Effective maximum is 127 (Byte.MAX_VALUE). We set the high order bit of the
+    // type byte in the WAL codecs to indicate, in a backwards compatible way, if the
+    // value is compressed there.
+    EffectiveMaximum((byte)Byte.MAX_VALUE),
+
     // Maximum is used when searching; you look from maximum on down.
     Maximum((byte)255);

Review comment:
       It is simpler if we don't try to conditionally compress values. No high bit twiddling. So there is no leakage into KeyValue in this case and no magic values.




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845354766


   Unit test failures are not related. It looks like master is recently unstable, related to RS groups. 


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634841914



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       This is done so we handle input as a sequence of byte[] , with ByteArrayInputStream providing the appropriate signals to the upper decompression stream as to when input is fully consumed. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630656411



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -241,10 +245,17 @@ public void write(Cell cell) throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.FAMILY));
       PrivateCellUtil.compressQualifier(out, cell,
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
-      // Write timestamp, type and value as uncompressed.
+      // Write timestamp, type and value.
       StreamUtils.writeLong(out, cell.getTimestamp());
-      out.write(cell.getTypeByte());
-      PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
+      byte type = cell.getTypeByte();

Review comment:
       Sure. This was left over from changes to make single step debugging easier. 




-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634042915



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();

Review comment:
       Any reason to do this here rather than in the c'tor?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+

Review comment:
       Do we need to call close() to cleanup any state with a CompressionContext tear down for custom compressed input streams?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));

Review comment:
       Same question as above.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {

Review comment:
       Perhaps explicitly mention in the javadoc this implementation is not thread-safe? 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       Why an extra copy here? Can't we just do lowerIn.setDelegate(in) ?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/io/DelegatingInputStream.java
##########
@@ -0,0 +1,99 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * An input stream that delegates all operations to another input stream.
+ * The delegate can be switched out for another at any time but to minimize the
+ * possibility of violating the InputStream contract it would be best to replace
+ * the delegate only once it has been fully consumed. <p> For example, a
+ * ByteArrayInputStream, which is implicitly bounded by the size of the underlying
+ * byte array can be converted into an unbounded stream fed by multiple instances
+ * of ByteArrayInputStream, switched out one for the other in sequence.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DelegatingInputStream extends InputStream {
+
+  InputStream lowerStream;
+
+  public DelegatingInputStream(InputStream lowerStream) {
+    this.lowerStream = lowerStream;
+  }
+
+  public InputStream getDelegate() {
+    return lowerStream;
+  }
+
+  public void setDelegate(InputStream lowerStream) {
+    this.lowerStream = lowerStream;
+  }
+
+  @Override
+  public int read() throws IOException {
+    return lowerStream.read();
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return lowerStream.read(b);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    return lowerStream.read(b, off, len);
+  }
+
+  @Override
+  public long skip(long n) throws IOException {
+    return lowerStream.skip(n);
+  }
+
+  @Override
+  public int available() throws IOException {
+    return lowerStream.available();
+  }
+
+  @Override
+  public void close() throws IOException {
+    lowerStream.close();
+  }
+
+  @Override
+  public synchronized void mark(int readlimit) {
+    lowerStream.mark(readlimit);
+  }
+
+  @Override
+  public synchronized void reset() throws IOException {
+    lowerStream.reset();
+  }
+
+  @Override
+  public boolean markSupported() {

Review comment:
       There is also a ProxyInputStream from apache.commons.io incase you want to reuse.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));
+        compressedIn = algorithm.createDecompressionStream(lowerIn, algorithm.getDecompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerIn.setDelegate(new ByteArrayInputStream(inBuffer));
+      }
+      return compressedIn.read(outArray, outOffset, outLength);

Review comment:
       I think we are relying on compressedIn being a buffered IS right? Otherwise read() may not return all the bytes?




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r628956646



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -220,6 +223,8 @@ public ByteString compress(byte[] data, Enum dictIndex) {
     }
   }
 
+  static final int VALUE_COMPRESS_THRESHOLD = 100;

Review comment:
       Based on my experimentation, zlib will compress a buffer of 100 bytes filled with zeros to 14 bytes. If we go by powers of 10, 100 seems fine, 10 seems not. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629759981



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +278,26 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      byte[] buffer = new byte[4096];
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      boolean finished = false;
+      do {
+        int bytesOut = deflater.deflate(buffer);
+        if (bytesOut > 0) {
+          baos.write(buffer, 0, bytesOut);
+        } else {
+          bytesOut = deflater.deflate(buffer, 0, buffer.length, Deflater.SYNC_FLUSH);

Review comment:
       I will add a comment here. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834993503


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   1m 47s |  root in master failed.  |
   | -1 :x: |  compile  |   0m 42s |  hbase-server in master failed.  |
   | +1 :green_heart: |  checkstyle  |   1m 35s |  master passed  |
   | -1 :x: |  spotbugs  |   0m 36s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 49s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  cc  |   0m 40s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 40s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  checkstyle  |   1m 33s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  hadoopcheck  |   1m 59s |  The patch causes 16 errors with Hadoop v3.1.2.  |
   | -1 :x: |  hadoopcheck  |   4m  2s |  The patch causes 16 errors with Hadoop v3.2.1.  |
   | -1 :x: |  hadoopcheck  |   6m  1s |  The patch causes 16 errors with Hadoop v3.3.0.  |
   | -1 :x: |  hbaseprotoc  |   0m 37s |  hbase-server in the patch failed.  |
   | -1 :x: |  spotbugs  |   0m 34s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 32s |  The patch does not generate ASF License warnings.  |
   |  |   |  34m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 1d7f3a118457 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 02b018cf1a |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/branch-compile-hbase-server.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/branch-spotbugs-hbase-server.txt |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | cc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-javac-3.1.2.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-javac-3.2.1.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-javac-3.3.0.txt |
   | hbaseprotoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-hbaseprotoc-hbase-server.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/artifact/yetus-general-check/output/patch-spotbugs-hbase-server.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/2/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-835946972


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   6m 39s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 52s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 25s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 43s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 31s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 27s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 27s |  the patch passed  |
   | +1 :green_heart: |  javac  |   5m 27s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 38s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  1s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  18m 13s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  0s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 38s |  The patch does not generate ASF License warnings.  |
   |  |   |  71m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 526e40bc729d 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8c2332d465 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/4/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840991114


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  5s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  4s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 139m 26s |  hbase-server in the patch passed.  |
   |  |   | 178m 34s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 6f9f68272f87 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / b65733c093 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/testReport/ |
   | Max. process+thread count | 3761 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] bharathv commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629772178



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -302,14 +343,28 @@ protected Cell parseCell() throws IOException {
         compression.getDictionary(CompressionContext.DictionaryIndex.QUALIFIER));
       pos += elemLen;
 
-      // timestamp, type and value
-      int tsTypeValLen = length - pos;
+      // timestamp
+      long ts = StreamUtils.readLong(in);
+      pos = Bytes.putLong(backingArray, pos, ts);
+      // type and value
+      int typeValLen = length - pos;
       if (tagsLength > 0) {
-        tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+        typeValLen = typeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
+      }
+      // high bit of type byte is 1 if value is compressed
+      byte type = (byte)in.read();
+      if ((type & 0x80) == 0x80) {

Review comment:
       Ya, my gut feeling says it will be amortized in a mix of small and large values and worst case assuming every value is small, the overhead is not that much (since its off by default anyway, people who choose to turn it on will experiment to see if it actually works for them). So for that is it worth complicating the code. Just throwing this idea out here.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
       "hbase.regionserver.wal.tags.enablecompression";
 
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+      "hbase.regionserver.wal.value.enablecompression";
+
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  static class ValueCompressor {
+    final Deflater deflater;
+    final Inflater inflater;
+
+    public ValueCompressor() {
+      deflater = new Deflater();
+      inflater = new Inflater();

Review comment:
       I'm talking about performance, not buffer overwrites. Following is the code path.
   
   [FS/Async]WAL#doAppend() -> writer.append() -> cellEncoder.Write() -> WallCellCodec.compressValue() -> deflater.deflate()
   
   all the deflate() calls are synchronized on the above `zsRef`. My question was does it affect the throughput of a wal appends?




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634843684



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];

Review comment:
       I added a comment in the source.




-- 
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.

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



[GitHub] [hbase] Apache-HBase removed a comment on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase removed a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834987575






-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634837582



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,20 +264,31 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private void writeCompressedValue(OutputStream out, Cell cell) throws IOException {
+      byte[] compressed = compression.getValueCompressor().compress(cell.getValueArray(),
+        cell.getValueOffset(), cell.getValueLength());
+      StreamUtils.writeRawVInt32(out, compressed.length);
+      out.write(compressed);
+    }
+
   }
 
   static class CompressedKvDecoder extends BaseDecoder {
     private final CompressionContext compression;
+    private final boolean hasValueCompression;
+    private final boolean hasTagCompression;
     public CompressedKvDecoder(InputStream in, CompressionContext compression) {
       super(in);
       this.compression = compression;
+      this.hasValueCompression = compression.hasValueCompression();
+      this.hasTagCompression = compression.hasTagCompression();
     }
 
     @Override
     protected Cell parseCell() throws IOException {

Review comment:
       No, compression is not supported in `SecureWALCellCodec`. 
   
   This is documented. You can have either compression or encryption, but not both. I'm sure this limitation can be addressed but would be separate follow up work.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r629779580



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       Sorry, these need to be public. 
   
       hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitValueCompression.java:[42,37]
       ENABLE_WAL_VALUE_COMPRESSION is not public in org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
       cannot be accessed from outside package




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840954177


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 20s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 24s |  the patch passed  |
   | +1 :green_heart: |  cc  |   5m 24s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 50s |  hbase-common generated 1 new + 158 unchanged - 1 fixed = 159 total (was 159)  |
   | -0 :warning: |  checkstyle  |   1m  5s |  hbase-server: The patch generated 1 new + 27 unchanged - 1 fixed = 28 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  18m 11s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  0s |  the patch passed  |
   | -1 :x: |  spotbugs  |   2m 17s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 36s |  The patch does not generate ASF License warnings.  |
   |  |   |  65m 59s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  org.apache.hadoop.hbase.regionserver.wal.CompressionContext$ValueCompressor.decompress(InputStream, int, byte[], int, int) ignores result of java.io.InputStream.read(byte[], int, int)  At CompressionContext.java:ignores result of java.io.InputStream.read(byte[], int, int)  At CompressionContext.java:[line 109] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 866245144e76 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / b65733c093 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/artifact/yetus-general-check/output/diff-compile-javac-hbase-common.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/artifact/yetus-general-check/output/whitespace-eol.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/12/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] ndimiduk commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-845371657


   > Unit test failures are not related. It looks like master is recently unstable, related to RS groups.
   
   That failure looks suspicious. Let me grab the logs before you merge/close the PR.


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837518016


   > Is it reasonable to always use the same compressor all the time for WALs? It strikes me that operators make an explicit choice about the compression used in their tables, so maybe they would expect that the compressor used by WALs would follow suit. I know it's not that simple, given that WALs are per host and not per table, so maybe it's something to consider in the future.
   
   If this is to be tied to table settings we would have to look up settings per table/CF whenever processing an edit. -1. I don't want to do this. It would be too expensive. 
   
   I see the goal here as making WALs not so terribly un-efficient to keep around. One pass of a standard compression over the values can achieve that, especially when we drop values that don't compress. Speed is important so BEST_SPEED is the right choice for compression level always. Enabling or disabling compression is enough configuration to give to users. This is what we have given them up to now. Now they can turn on or off value compression too. Arguably zlib compression level here is not a user serviceable setting but I could be convinced to at least make that a site configuration setting. 


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837666043


   I have some changes coming soon. Accepted @bharathv 's argument that unconditionally compressing values if value compression is enabled is fine even if some value cases may not compress because gains and losses are amortized over all values in the WAL file. 
   
   Also addresses several points of @ndimiduk 's feedback by mooting the discussion. 


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-841585645


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 25s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 15s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 10s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  6s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 141m 40s |  hbase-server in the patch passed.  |
   |  |   | 181m 34s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 89e23d915d87 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8ae4d65aa5 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/testReport/ |
   | Max. process+thread count | 3932 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/13/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-838765223


   There was a period of time where there was a bad patch. I pushed a replacement. The latest precommits look good. 


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-841533295


   > Thanks for the detailed perf results, SNAPPY seems like the sweet spot. Avg append time took a good hit? (10x IIUC?), something to be concerned about?
   
   The extra latency is going to be very much workload dependent, and so WAL value compression should remain a non-default configuration option.
   
   The test case I used to produce this benchmark employs very large values. We are loading in entries from Common Crawl -- web pages, images, javascript, CSS, big xml. So, that is both the source of the good compression results -- the values are large and compressible, generally -- and the source of the extra WAL append latency -- the values are large, generally, so the compression codecs have a fair amount of work. Even so, SNAPPY only adding ~2ms per append under these circumstances is a great result. 
   
   For smaller values the latency hit will be proportionally lesser. 
   
   > Btw, is this ready for review or are you still iterating on the code?
   
   See the findbugs warning above and just assume I will fix that. Otherwise I'm waiting for further feedback. 
   
   @bharathv 


-- 
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.

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



[GitHub] [hbase] apurtell edited a comment on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell edited a comment on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-838973994


   > Speaking out loud. During RU, the wal written by RS that has ENABLE_WAL_VALUE_COMPRESSION set to false died and another RS which was recently upgraded has the above property set to true. Will the new RS be able to decode the cell ?
   
   @shahrs87  Yes. 
   
   Each WAL file carries metadata about compression particulars for the file, so we can set up the CompressionContext correctly no matter the site file defaults. This logic is in ProtobufLogReader.java. 
   
   The inverse is not true, though. There is no downgrade scenario for when after an upgrade now new WALs have value compression enabled, and you want to downgrade to a version that does not support value compression. However this is the obvious expectation that comes with enabling a new optional feature, and will be documented in release notes.


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630330429



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       Ah, I see what you mean. Never mind. Let me fix. 




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634845897



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));
+        compressedIn = algorithm.createDecompressionStream(lowerIn, algorithm.getDecompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerIn.setDelegate(new ByteArrayInputStream(inBuffer));
+      }
+      return compressedIn.read(outArray, outOffset, outLength);

Review comment:
       > Actual bytes read are returned
   
   I added an assertion to this effect in `WALCellCodec#readCompressedValue`




-- 
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.

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



[GitHub] [hbase] apurtell commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834951042


   Clean up the commit message.


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-840115789


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 41s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 23s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 15s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 17s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 17s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m 56s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 58s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 11s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 239m 56s |  hbase-server in the patch passed.  |
   |  |   | 282m 46s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/11/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b17c647991a4 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 630c73fda4 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/11/testReport/ |
   | Max. process+thread count | 3317 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/11/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r628787521



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +270,28 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    public static void writeCompressedValue(OutputStream out, byte[] valueArray, int offset,
+        int vlength, Deflater deflater) throws IOException {
+      byte[] buffer = new byte[4096];
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      deflater.reset();
+      deflater.setInput(valueArray, offset, vlength);
+      boolean finished = false;
+      do {
+        int bytesOut = deflater.deflate(buffer);
+        if (bytesOut > 0) {
+          baos.write(buffer, 0, bytesOut);
+        } else {
+          bytesOut = deflater.deflate(buffer, 0, buffer.length, Deflater.FULL_FLUSH);

Review comment:
       Currently we completely flush the encoder at the end of every value (FULL_FLUSH). This is very conservative, allowing each value to be decompressed individually (so is resilient to corruptions), but impacts compression ratio. Meanwhile our custom dictionary scheme accumulates strings over the whole file, so we are being conservative in value compression in a way our custom scheme defeats anyway. We can instead also let the Deflater build its dictionary over all values in the WAL, using SYNC_FLUSH instead. 
   
   Note then we can't create a Deflater for each value in `readCompressedValue`, we have to init one in the `CompressionContext` when we set up the context and reuse it, so it can build its dictionary in the same way as the deflater did, over all values in the file. 




-- 
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.

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



[GitHub] [hbase] ndimiduk commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-837437980


   Oh, one other comment I forgot to mention before hitting submit. Is it reasonable to always use the same compressor all the time for WALs? It strikes me that operators make an explicit choice about the compression used in their tables, so maybe they would expect that the compressor used by WALs would follow suit. I know it's not that simple, given that WALs are per host and not per table, so maybe it's something to consider in the future.


-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-844612699


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  10m  6s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  6s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m 53s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 30s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 23s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |  10m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 16s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 29s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 226m 58s |  hbase-server in the patch passed.  |
   |  |   | 285m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 110bacffde07 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fe47557f4c |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/testReport/ |
   | Max. process+thread count | 2982 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/15/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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



[GitHub] [hbase] ndimiduk commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630591194



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -34,21 +36,49 @@
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =

Review comment:
       Oh -- it seems that `TestWALSplitValueCompression` is in the wrong package :)
   
   Not related to your change, and I found several similar cases to this while closing down symbol permissions in other areas.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r634840135



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
##########
@@ -18,37 +18,117 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.io.DelegatingInputStream;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
+import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public class CompressionContext {
 
-  static final String ENABLE_WAL_TAGS_COMPRESSION =
-      "hbase.regionserver.wal.tags.enablecompression";
+  public static final String ENABLE_WAL_TAGS_COMPRESSION =
+    "hbase.regionserver.wal.tags.enablecompression";
+
+  public static final String ENABLE_WAL_VALUE_COMPRESSION =
+    "hbase.regionserver.wal.value.enablecompression";
+
+  public static final String WAL_VALUE_COMPRESSION_TYPE =
+    "hbase.regionserver.wal.value.compression.type";
 
   public enum DictionaryIndex {
     REGION, TABLE, FAMILY, QUALIFIER, ROW
   }
 
+  /**
+   * Encapsulates the compression algorithm and its streams that we will use for value
+   * compression in this WAL.
+   */
+  static class ValueCompressor {
+  
+    static final int IO_BUFFER_SIZE = 4096;
+
+    private final Compression.Algorithm algorithm;
+    private DelegatingInputStream lowerIn;
+    private ByteArrayOutputStream lowerOut;
+    private InputStream compressedIn;
+    private OutputStream compressedOut;
+
+    public ValueCompressor(Compression.Algorithm algorithm) throws IOException {
+      this.algorithm = algorithm;
+    }
+
+    public Compression.Algorithm getAlgorithm() {
+      return algorithm;
+    }
+
+    public byte[] compress(byte[] valueArray, int valueOffset, int valueLength)
+        throws IOException {
+      // We have to create the output streams here the first time around.
+      if (compressedOut == null) {
+        lowerOut = new ByteArrayOutputStream();
+        compressedOut = algorithm.createCompressionStream(lowerOut, algorithm.getCompressor(),
+          IO_BUFFER_SIZE);
+      } else {
+        lowerOut.reset();
+      }
+      compressedOut.write(valueArray, valueOffset, valueLength);
+      compressedOut.flush();
+      return lowerOut.toByteArray();
+    }
+
+    public int decompress(InputStream in, int inLength, byte[] outArray, int outOffset,
+        int outLength) throws IOException {
+      // Read all of the compressed bytes into a buffer.
+      byte[] inBuffer = new byte[inLength];
+      IOUtils.readFully(in, inBuffer);
+      // We have to create the input streams here the first time around.
+      if (compressedIn == null) {
+        lowerIn = new DelegatingInputStream(new ByteArrayInputStream(inBuffer));

Review comment:
       I forget exactly which case it was, either input or output, but in one case we can't set up until we have the user supplied stream, because an IO action is immediately taken. So I do it for both read and write steams to keep the code congruent. 




-- 
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.

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



[GitHub] [hbase] shahrs87 commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
shahrs87 commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630264621



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       @apurtell  I was trying to understand how this buffer works while we compress the data. So I wrote a small test case to understand it better.
   
   ```
     @Test
     public void testCodec() throws Exception {
       String inputRawString = "helloworld";
       int rawStringLength = inputRawString.length();
       byte[] compressedBytesArr = compressValue(inputRawString);
       int compressedBytesLength = compressedBytesArr.length;
     }
   
     private byte[] compressValue(String inputRawString) throws IOException {
       Deflater deflater = new Deflater();
       deflater.setLevel(Deflater.BEST_SPEED);
   
       byte[] input = inputRawString.getBytes("UTF-8");
       deflater.setInput(input, 0, input.length);
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       byte[] buffer = new byte[5];
   
       int bytesOut;
       do {
         bytesOut = deflater.deflate(buffer);
         if (bytesOut > 0) {
           baos.write(buffer, 0, bytesOut);
         }
       } while (bytesOut > 0);
       // Done compressing value, now flush until deflater buffers are empty
       do {
         bytesOut = deflater.deflate(buffer, 0, buffer.length, Deflater.SYNC_FLUSH);
         baos.write(buffer, 0, bytesOut);
       } while (bytesOut == buffer.length); // See javadoc for Deflater#deflate
       return baos.toByteArray();
     }
   ```
   
   Basically the above snippet is copied pasted from your PR where you compress the cell. I replcaed the cell with some random String.
   Also I made sure that the size of buffer (5) is less than input string length (which is 10) since in real world the size of Cell value will be more than 1024 which is the buffer size in PR.
   The above test goes into indefinite while loop in the second do..while loop.
   If I change the size of buffer to 10 then it succeeds.
   Please educate me if I am doing something wrong.




-- 
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.

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



[GitHub] [hbase] apurtell commented on a change in pull request #3244: HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#discussion_r630330429



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
##########
@@ -256,6 +267,43 @@ public void write(Cell cell) throws IOException {
         }
       }
     }
+
+    private byte[] compressValue(Cell cell) throws IOException {
+      Deflater deflater = compression.getValueCompressor().getDeflater();
+      if (cell instanceof ByteBufferExtendedCell) {
+        deflater.setInput(((ByteBufferExtendedCell)cell).getValueByteBuffer().array(),
+          ((ByteBufferExtendedCell)cell).getValueByteBuffer().arrayOffset() +
+          ((ByteBufferExtendedCell)cell).getValuePosition(),
+          cell.getValueLength());
+      } else {
+        deflater.setInput(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+      }
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       Ah, I see what you mean. Never mind. Let me fix. 




-- 
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.

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



[GitHub] [hbase] Apache-HBase commented on pull request #3244: [WIP] HBASE-25869 WAL value compression

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3244:
URL: https://github.com/apache/hbase/pull/3244#issuecomment-834961724


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | -1 :x: |  mvninstall  |   2m 22s |  root in master failed.  |
   | -1 :x: |  compile  |   0m 41s |  hbase-server in master failed.  |
   | -1 :x: |  shadedjars  |   6m 11s |  branch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   2m  5s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 41s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 41s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   6m  3s |  patch has 16 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 57s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  4s |  hbase-common in the patch passed.  |
   | -1 :x: |  unit  |   0m 41s |  hbase-server in the patch failed.  |
   |  |   |  29m 46s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3244 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d034c6a3d124 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 02b018cf1a |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/branch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/branch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/branch-shadedjars.txt |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/testReport/ |
   | Max. process+thread count | 275 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3244/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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.

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