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 2020/12/08 10:15:11 UTC

[GitHub] [hbase] ramkrish86 opened a new pull request #2747: Branch 2.3 hbase 24850

ramkrish86 opened a new pull request #2747:
URL: https://github.com/apache/hbase/pull/2747


   Closed the original PR due to some issues with my linux/windows environment toggling. Created a new PR which can compile too. 
   This version of the patch tries to introduce an interface ContiguousCellFormat which understands the KV format where the data is arranged in the KV serialization format.
   It tries to minimize the branching in cases of pure Kv or pure ByteBufferKV. with this patch and JMH like test with adding >100MB of data getting added to Memstore like CSLM provides >50% improvement where all the cells are pure KVs.
   
   We did some cluster testing with only KV as the cell type and also with no DBEs. We might need some more tests to ensure we don't break anything.
   In this commit apart from having the ContiguousCellComparator, We also found that the bulk load performance was slower inspite of overall improving the comparator performance by above 15%.
   The reason was that PutsortReducer - get a given row with all the cells for that row and that gets written to the hfile. So effectively it is one row that is geting added to the map. Now even when cases where there are 300 cells in a row, the optimization that we expect out of ContiguousCellComparator changes does not kick in. That is due to the various branches we still have in the code and the number of cells for the optimization to kick in is still lesser.
   For those cases if we can bring up the KVComparator again (currently it is deprecated - see the PutsortReducer changes in the patch) and use that KVComparator specifically for these bulk load type of cases then we are performing 15% faster than 1.3 branch. This is in line with what we are trying to do in https://issues.apache.org/jira/browse/HBASE-24754.
   I can open up a discussion thread with all the details in the dev@ for others to chime in.
   @anoopsjohn , @saintstack - FYI.
   


----------------------------------------------------------------
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] saintstack commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ContiguousCellFormat.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * The interface represents any cell impl that is backed by contiguous layout like {@link KeyValue},
+ * {@link ByteBufferKeyValue}. The APIs in this interface helps to easily access the internals of
+ * the Cell structure so that we avoid fetching the offsets and lengths every time from the internal
+ * backed memory structure. We can use this interface only if it follows the standard KeyValue
+ * format. Helps in performance and this is jit friendly
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ContiguousCellFormat {
+
+  /**
+   * Return the family length position/offset.
+   * @param rowLen - the rowlength of the cell
+   * @return the family's offset position
+   */
+  int getFamilyLengthPosition(int rowLen);
+
+  /**
+   * Return the family length. Use this along with {@link #getFamilyLengthPosition(int)}
+   * @param famLengthPosition - the family offset
+   * @return the family's length
+   */
+  byte getFamilyLength(int famLengthPosition);
+
+  /**
+   * Return the family position/offset. This can be used along with
+   * the result of {@link #getFamilyLengthPosition(int)}
+   * @param familyLengthPosition - the family length position
+   * @return the family's position
+   */
+  int getFamilyInternalPosition(int familyLengthPosition);
+
+  /**
+   * Return the qualifier position/offset
+   * @param famOffset - the family offset
+   * @param famLength - the family length
+   * @return the qualifier offset/position.
+   */
+  int getQualifierInternalPosition(int famOffset, byte famLength);
+
+  /**
+   * Return the qualifier length
+   * @param keyLength - the key length
+   * @param rowLen - the row length
+   * @param famLength - the family length
+   * @return the qualifier length
+   */
+  int getQualifierLength(int keyLength, int rowLen, int famLength);
+
+  /**
+   * Return the time stamp. Use this along with {@link #getKeyLength()}
+   * @param keyLength - the key length
+   * @return the timestamp
+   */
+  long getTimestamp(int keyLength);
+
+  /**
+   * Return the type byte. Use this along with {@link #getKeyLength()}
+   * @param keyLength - the key length
+   * @return - the type byte
+   */
+  byte getTypeByte(int keyLength);
+
+  /**
+   * The keylength of the cell
+   * @return the key length
+   */
+  int getKeyLength();
+}

Review comment:
       nod




----------------------------------------------------------------
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 #2747: HBASE-24850 CellComparator perf improvement

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






----------------------------------------------------------------
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] saintstack commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {

Review comment:
       Suggestion.
   
   Main point is worry about exposing implementation detail especially after all the work you lot did to change the KV so we could put another impl in place (for example, one that caches sizes as they are calculated?)




----------------------------------------------------------------
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] ramkrish86 commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {

Review comment:
       I thought of this static utility class . So something like moving this entire code into a Util class? 
   >>For instance, what if a Cell implementation had data members that cached all lengths.
   This we did not do and always we restrain from doing this because say if we add those cells to memstore most of the size will go for the cell overhead. Say in a 128M flush may be 80M only is data remaining may become only overhead. 




----------------------------------------------------------------
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] ramkrish86 commented on pull request #2747: HBASE-24850 CellComparator perf improvement

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


   `A Cell implementation that does lazy length caching internally? Then we'd not need comparator knowing about implementation? What you think Ram?`
   Caching everything internally will add to the overhead of the cell. That we were always restrictive and I think it is better to be so. We still do some of that in SizeCachedKV impl of the cell. But for KVs which is mostly at the memstore layer that is going to cost us in how much we 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] ramkrish86 commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ContiguousCellFormat.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * The interface represents any cell impl that is backed by contiguous layout like {@link KeyValue},
+ * {@link ByteBufferKeyValue}. The APIs in this interface helps to easily access the internals of
+ * the Cell structure so that we avoid fetching the offsets and lengths every time from the internal
+ * backed memory structure. We can use this interface only if it follows the standard KeyValue
+ * format. Helps in performance and this is jit friendly
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ContiguousCellFormat {
+
+  /**
+   * Return the family length position/offset.
+   * @param rowLen - the rowlength of the cell
+   * @return the family's offset position
+   */
+  int getFamilyLengthPosition(int rowLen);
+
+  /**
+   * Return the family length. Use this along with {@link #getFamilyLengthPosition(int)}
+   * @param famLengthPosition - the family offset
+   * @return the family's length
+   */
+  byte getFamilyLength(int famLengthPosition);
+
+  /**
+   * Return the family position/offset. This can be used along with
+   * the result of {@link #getFamilyLengthPosition(int)}
+   * @param familyLengthPosition - the family length position
+   * @return the family's position
+   */
+  int getFamilyInternalPosition(int familyLengthPosition);
+
+  /**
+   * Return the qualifier position/offset
+   * @param famOffset - the family offset
+   * @param famLength - the family length
+   * @return the qualifier offset/position.
+   */
+  int getQualifierInternalPosition(int famOffset, byte famLength);
+
+  /**
+   * Return the qualifier length
+   * @param keyLength - the key length
+   * @param rowLen - the row length
+   * @param famLength - the family length
+   * @return the qualifier length
+   */
+  int getQualifierLength(int keyLength, int rowLen, int famLength);
+
+  /**
+   * Return the time stamp. Use this along with {@link #getKeyLength()}
+   * @param keyLength - the key length
+   * @return the timestamp
+   */
+  long getTimestamp(int keyLength);
+
+  /**
+   * Return the type byte. Use this along with {@link #getKeyLength()}
+   * @param keyLength - the key length
+   * @return - the type byte
+   */
+  byte getTypeByte(int keyLength);
+
+  /**
+   * The keylength of the cell
+   * @return the key length
+   */
+  int getKeyLength();
+}

Review comment:
       `It doesn't look like it. Has to be exposed so Comparator can make use of these methods. I seen that this patch is just generalizing the behavior done when we added BBKVComparator. Argh.`
   Yes I agree. I wanted to make it generic. Also the fact that if we have KV and BBKV in my cells I can get similar performance with this patch because we are having the internals with us. 




----------------------------------------------------------------
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] saintstack commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {
+    return this.offset + Bytes.SIZEOF_SHORT + rowLen;
+  }
+
+  @Override
+  public int getFamilyInternalPosition(int familyLengthPosition) {
+    return familyLengthPosition + Bytes.SIZEOF_BYTE;
+  }
+
+  @Override
+  public int getQualifierInternalPosition(int famOffset, byte famLength) {
+    return famOffset + famLength;
+  }
+
+  private int getTimestampOffset(final int keylength) {
+    return this.offset + keylength - KeyValue.TIMESTAMP_TYPE_SIZE;
+  }
+
+  @Override
+  public long getTimestamp(int keyLength) {
+    int tsOffset = getTimestampOffset(keyLength);
+    return ByteBufferUtils.toLong(this.buf, tsOffset);
+  }
+
+  @Override
+  public byte getTypeByte(int keyLength) {
+    return ByteBufferUtils.toByte(this.buf, this.offset + keyLength - 1);
+  }
+
+  @Override
+  public int getKeyLength() {
+    return this.length;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return super.equals(other);
+  }
+
+  @Override
+  public int hashCode() {

Review comment:
       ok




----------------------------------------------------------------
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] saintstack commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -161,11 +161,11 @@ private int getTimestampOffset() {
 
   @Override
   public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
+    return getTypeByte(this.length);

Review comment:
       Ok. Maybe if a new patch, rename this data member to keyLength?




----------------------------------------------------------------
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] saintstack commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -161,11 +161,11 @@ private int getTimestampOffset() {
 
   @Override
   public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
+    return getTypeByte(this.length);

Review comment:
       this.length is updated as we parse? Presumes we read Cell parts in order... first the row and then the CF and then the qualifier? We are not allowed to just and read the qualifier w/o first reading row?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -161,11 +161,11 @@ private int getTimestampOffset() {
 
   @Override
   public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
+    return getTypeByte(this.length);
   }
 
   @Override
-  public void setSequenceId(long seqId) throws IOException {
+  public void setSequenceId(long seqId) {
     throw new IllegalArgumentException("This is a key only Cell");

Review comment:
       Good.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {

Review comment:
       This stuff used to be private. Making it public exposes the implementation. You fellows did a mountain of work making it so we could CHANGE the implementation. This goes back on that work? At least exposing this stuff as public methods?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {
+    return this.offset + Bytes.SIZEOF_SHORT + rowLen;
+  }
+
+  @Override
+  public int getFamilyInternalPosition(int familyLengthPosition) {
+    return familyLengthPosition + Bytes.SIZEOF_BYTE;
+  }
+
+  @Override
+  public int getQualifierInternalPosition(int famOffset, byte famLength) {
+    return famOffset + famLength;
+  }
+
+  private int getTimestampOffset(final int keylength) {
+    return this.offset + keylength - KeyValue.TIMESTAMP_TYPE_SIZE;
+  }
+
+  @Override
+  public long getTimestamp(int keyLength) {
+    int tsOffset = getTimestampOffset(keyLength);
+    return ByteBufferUtils.toLong(this.buf, tsOffset);
+  }
+
+  @Override
+  public byte getTypeByte(int keyLength) {
+    return ByteBufferUtils.toByte(this.buf, this.offset + keyLength - 1);
+  }
+
+  @Override
+  public int getKeyLength() {
+    return this.length;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return super.equals(other);
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
 }

Review comment:
       Have you run w/ the jdk compile flags on to make sure all methods doing compare get inlined? I found it informative... sometimes methods would not inline or compile because too big....  Sometimes fixing this w/ refactor helped improve perf.
   
   What about that nice stack trace you showed in the issue where you showed deep trace for hbase2 in compare but a shallow one for hbase1. As you suggested in the JIRA, yes, a deep stack trace costs... trimming it would help perf too.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -51,35 +53,38 @@
    */
   public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl();
 
+  private static final ContiguousCellFormatComparator contiguousCellComparator =

Review comment:
       Hmm... when would a comparator NOT be do left-to-right?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -76,7 +77,7 @@
  * length and actual tag bytes length.
  */
 @InterfaceAudience.Private
-public class KeyValue implements ExtendedCell, Cloneable {
+public class KeyValue implements ExtendedCell, ContiguousCellFormat, Cloneable {

Review comment:
       Could the methods be added to ExtendedCell or is that different concerns?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -51,35 +53,38 @@
    */
   public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl();
 
+  private static final ContiguousCellFormatComparator contiguousCellComparator =

Review comment:
       Won't it always be a 'contiguous' left-to-right compare?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {
+    return this.offset + Bytes.SIZEOF_SHORT + rowLen;
+  }
+
+  @Override
+  public int getFamilyInternalPosition(int familyLengthPosition) {
+    return familyLengthPosition + Bytes.SIZEOF_BYTE;
+  }
+
+  @Override
+  public int getQualifierInternalPosition(int famOffset, byte famLength) {
+    return famOffset + famLength;
+  }
+
+  private int getTimestampOffset(final int keylength) {
+    return this.offset + keylength - KeyValue.TIMESTAMP_TYPE_SIZE;
+  }
+
+  @Override
+  public long getTimestamp(int keyLength) {
+    int tsOffset = getTimestampOffset(keyLength);
+    return ByteBufferUtils.toLong(this.buf, tsOffset);
+  }
+
+  @Override
+  public byte getTypeByte(int keyLength) {
+    return ByteBufferUtils.toByte(this.buf, this.offset + keyLength - 1);
+  }
+
+  @Override
+  public int getKeyLength() {
+    return this.length;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return super.equals(other);

Review comment:
       Remove?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
##########
@@ -20,41 +20,128 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.List;
+
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
 
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
  * off heap/ on heap ByteBuffer
  */
 @InterfaceAudience.Private
-public class ByteBufferKeyValue extends ByteBufferExtendedCell {
+public class ByteBufferKeyValue extends ByteBufferExtendedCell implements ContiguousCellFormat {

Review comment:
       I think that to introduce a new Interface here around Cells, you first need to purge two old ones... smile. Ask @anoopsjohn 

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
##########
@@ -20,41 +20,128 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.List;
+
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
 
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
  * off heap/ on heap ByteBuffer
  */
 @InterfaceAudience.Private
-public class ByteBufferKeyValue extends ByteBufferExtendedCell {
+public class ByteBufferKeyValue extends ByteBufferExtendedCell implements ContiguousCellFormat {
 
-  protected final ByteBuffer buf;

Review comment:
       Why take away the final?
   
   If for default constructor, pass nulls to the override?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {
+    return this.offset + Bytes.SIZEOF_SHORT + rowLen;
+  }
+
+  @Override
+  public int getFamilyInternalPosition(int familyLengthPosition) {
+    return familyLengthPosition + Bytes.SIZEOF_BYTE;
+  }
+
+  @Override
+  public int getQualifierInternalPosition(int famOffset, byte famLength) {
+    return famOffset + famLength;
+  }
+
+  private int getTimestampOffset(final int keylength) {
+    return this.offset + keylength - KeyValue.TIMESTAMP_TYPE_SIZE;
+  }
+
+  @Override
+  public long getTimestamp(int keyLength) {
+    int tsOffset = getTimestampOffset(keyLength);
+    return ByteBufferUtils.toLong(this.buf, tsOffset);
+  }
+
+  @Override
+  public byte getTypeByte(int keyLength) {
+    return ByteBufferUtils.toByte(this.buf, this.offset + keyLength - 1);
+  }
+
+  @Override
+  public int getKeyLength() {
+    return this.length;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return super.equals(other);
+  }
+
+  @Override
+  public int hashCode() {

Review comment:
       Can remove these?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -51,35 +53,38 @@
    */
   public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl();
 
+  private static final ContiguousCellFormatComparator contiguousCellComparator =
+      new ContiguousCellFormatComparator(COMPARATOR);
+
   @Override
   public final int compare(final Cell a, final Cell b) {
     return compare(a, b, false);
   }
 
   @Override
   public int compare(final Cell a, final Cell b, boolean ignoreSequenceid) {
-
     int diff = 0;
-    // "Peel off" the most common path.
-    if (a instanceof ByteBufferKeyValue && b instanceof ByteBufferKeyValue) {
-      diff = BBKVComparator.compare((ByteBufferKeyValue)a, (ByteBufferKeyValue)b, ignoreSequenceid);
-      if (diff != 0) {
-        return diff;
-      }
+    // "Peeling off" the most common cases where the Cells backed by KV format either onheap or
+    // offheap
+    if (a instanceof ContiguousCellFormat && b instanceof ContiguousCellFormat

Review comment:
       I see. You want to use a comparator that has expectations about internals... that there methods it can call to speed up the compare.
   
   Man. We have too many if/else's in the path. if BB, if tags, if sequenceid, if offheap.... if unsafe. If ByteBufferExtendedCell...
   
   We don't yet have an implementation that is not contiguous?
   
   

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -161,11 +161,11 @@ private int getTimestampOffset() {
 
   @Override
   public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
+    return getTypeByte(this.length);

Review comment:
       But maybe this.length doesn't change? Its the key length?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ContiguousCellFormat.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * The interface represents any cell impl that is backed by contiguous layout like {@link KeyValue},
+ * {@link ByteBufferKeyValue}. The APIs in this interface helps to easily access the internals of
+ * the Cell structure so that we avoid fetching the offsets and lengths every time from the internal
+ * backed memory structure. We can use this interface only if it follows the standard KeyValue
+ * format. Helps in performance and this is jit friendly
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ContiguousCellFormat {
+
+  /**
+   * Return the family length position/offset.
+   * @param rowLen - the rowlength of the cell
+   * @return the family's offset position
+   */
+  int getFamilyLengthPosition(int rowLen);
+
+  /**
+   * Return the family length. Use this along with {@link #getFamilyLengthPosition(int)}
+   * @param famLengthPosition - the family offset
+   * @return the family's length
+   */
+  byte getFamilyLength(int famLengthPosition);
+
+  /**
+   * Return the family position/offset. This can be used along with
+   * the result of {@link #getFamilyLengthPosition(int)}
+   * @param familyLengthPosition - the family length position
+   * @return the family's position
+   */
+  int getFamilyInternalPosition(int familyLengthPosition);
+
+  /**
+   * Return the qualifier position/offset
+   * @param famOffset - the family offset
+   * @param famLength - the family length
+   * @return the qualifier offset/position.
+   */
+  int getQualifierInternalPosition(int famOffset, byte famLength);
+
+  /**
+   * Return the qualifier length
+   * @param keyLength - the key length
+   * @param rowLen - the row length
+   * @param famLength - the family length
+   * @return the qualifier length
+   */
+  int getQualifierLength(int keyLength, int rowLen, int famLength);
+
+  /**
+   * Return the time stamp. Use this along with {@link #getKeyLength()}
+   * @param keyLength - the key length
+   * @return the timestamp
+   */
+  long getTimestamp(int keyLength);
+
+  /**
+   * Return the type byte. Use this along with {@link #getKeyLength()}
+   * @param keyLength - the key length
+   * @return - the type byte
+   */
+  byte getTypeByte(int keyLength);
+
+  /**
+   * The keylength of the cell
+   * @return the key length
+   */
+  int getKeyLength();
+}

Review comment:
       Could this not just be internal to the Cell implementation? Does it have to be exposed like this in an Interface with special comparator?
   
   It doesn't look like it. Has to be exposed so Comparator can make use of these methods. I seen that this patch is just generalizing the behavior done when we added BBKVComparator. Argh.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {

Review comment:
       For instance, what if a Cell implementation had data members that cached all lengths... a column family length data member and a row length data member, etc. These methods wouldn't make sense to it?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {

Review comment:
       What if this stuff just stayed internal? I think you said, I did the KV one and here you are doing BB. Do we have to do more? There'd be duplication... or we could call out to a utility class of statics so the offset math could be shared.... 




----------------------------------------------------------------
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 #2747: HBASE-24850 CellComparator perf improvement

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 17s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  6s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2.3 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 31s |  branch-2.3 passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  branch-2.3 passed  |
   | +1 :green_heart: |  shadedjars  |   5m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 41s |  branch-2.3 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 13s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 50s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 50s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-common generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 28s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |  11m 46s |  hbase-mapreduce in the patch passed.  |
   |  |   |  36m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-jdk8-hadoop2-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2747 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e7d5a2c89ada 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2.3 / 7b1e3e9268 |
   | Default Java | AdoptOpenJDK-1.8.0_232-b09 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-jdk8-hadoop2-check/output/diff-javadoc-javadoc-hbase-common.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/testReport/ |
   | Max. process+thread count | 3551 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-mapreduce U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/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] saintstack closed pull request #2747: HBASE-24850 CellComparator perf improvement

Posted by GitBox <gi...@apache.org>.
saintstack closed pull request #2747:
URL: https://github.com/apache/hbase/pull/2747


   


----------------------------------------------------------------
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] saintstack commented on pull request #2747: HBASE-24850 CellComparator perf improvement

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


   @ramkrish86 Should we close this in favor of #2776  ?


----------------------------------------------------------------
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] saintstack commented on pull request #2747: HBASE-24850 CellComparator perf improvement

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


   @ramkrish86  Did you get a chance to read through the above (and then close this)? Thanks.


----------------------------------------------------------------
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] ramkrish86 commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -51,35 +53,38 @@
    */
   public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl();
 
+  private static final ContiguousCellFormatComparator contiguousCellComparator =
+      new ContiguousCellFormatComparator(COMPARATOR);
+
   @Override
   public final int compare(final Cell a, final Cell b) {
     return compare(a, b, false);
   }
 
   @Override
   public int compare(final Cell a, final Cell b, boolean ignoreSequenceid) {
-
     int diff = 0;
-    // "Peel off" the most common path.
-    if (a instanceof ByteBufferKeyValue && b instanceof ByteBufferKeyValue) {
-      diff = BBKVComparator.compare((ByteBufferKeyValue)a, (ByteBufferKeyValue)b, ignoreSequenceid);
-      if (diff != 0) {
-        return diff;
-      }
+    // "Peeling off" the most common cases where the Cells backed by KV format either onheap or
+    // offheap
+    if (a instanceof ContiguousCellFormat && b instanceof ContiguousCellFormat

Review comment:
       `We have too many if/else's in the path. if BB, if tags, if sequenceid, if offheap.... if unsafe. If ByteBufferExtendedCell...`
   
   Previously we had all these branches for BytebufferExtendedcell and left being normal cell and right being normal cell etc. But we were not having the optimization of knowing the internals. Now this branching is fixed and the branches are only at this place and internally no branches. Previously the compareRows() had 4 branches, compareFamilies() had 4 branches, comparequalifiers() had 4 branches and then the tags and sequenceId. This reduces those branches. I strongly favour this lesser branching but more duplicate 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] ramkrish86 commented on pull request #2747: HBASE-24850 CellComparator perf improvement

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


   Sure Stack let me close this. But before that I will ensure I read all your comments and address them in the other PR including some profiling. 


----------------------------------------------------------------
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 #2747: HBASE-24850 CellComparator perf improvement

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 33s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +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.  |
   ||| _ branch-2.3 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  7s |  branch-2.3 passed  |
   | +1 :green_heart: |  checkstyle  |   0m 49s |  branch-2.3 passed  |
   | +1 :green_heart: |  spotbugs  |   1m 38s |  branch-2.3 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 23s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 31s |  hbase-common: The patch generated 2 new + 192 unchanged - 1 fixed = 194 total (was 193)  |
   | -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  |  23m 58s |  Patch does not cause any errors with Hadoop 2.10.0 or 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 16s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 26s |  The patch does not generate ASF License warnings.  |
   |  |   |  51m 45s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2747 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 2db23ca39cc0 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 | branch-2.3 / 7b1e3e9268 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/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-2747/2/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-mapreduce U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=3.1.12 |
   | 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] ramkrish86 commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -51,35 +53,38 @@
    */
   public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl();
 
+  private static final ContiguousCellFormatComparator contiguousCellComparator =

Review comment:
       the term Contiguous was added in terms of the Kv format. In case of cells that are encoder based it is alwasy from left to right but the format may not be KV based. Hence I went with the word 'contiguous'.




----------------------------------------------------------------
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] saintstack commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -51,35 +53,38 @@
    */
   public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl();
 
+  private static final ContiguousCellFormatComparator contiguousCellComparator =

Review comment:
       Haven't checked but might be worth adding this note on what contiguous means to your marker interface....




----------------------------------------------------------------
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 #2747: HBASE-24850 CellComparator perf improvement

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  7s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2.3 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  branch-2.3 passed  |
   | +1 :green_heart: |  compile  |   0m 53s |  branch-2.3 passed  |
   | +1 :green_heart: |  shadedjars  |   5m 49s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in branch-2.3 failed.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-mapreduce in branch-2.3 failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 55s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 55s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 48s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-mapreduce in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 46s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m 48s |  hbase-mapreduce in the patch passed.  |
   |  |   |  36m 31s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2747 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a92c97eeedfc 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 | branch-2.3 / 7b1e3e9268 |
   | Default Java | AdoptOpenJDK-11.0.6+10 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-mapreduce.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-mapreduce.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/2/testReport/ |
   | Max. process+thread count | 4953 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-mapreduce U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/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] ramkrish86 commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -161,11 +161,11 @@ private int getTimestampOffset() {
 
   @Override
   public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
+    return getTypeByte(this.length);

Review comment:
       In this case it is the keylength only because it is BbKeyOnlyKV.




----------------------------------------------------------------
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] ramkrish86 commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -76,7 +77,7 @@
  * length and actual tag bytes length.
  */
 @InterfaceAudience.Private
-public class KeyValue implements ExtendedCell, Cloneable {
+public class KeyValue implements ExtendedCell, ContiguousCellFormat, Cloneable {

Review comment:
       I think we need not add to ExtendedCell. ExtendedCell is for server to create cells. This new interface only the comparator can understand. 




----------------------------------------------------------------
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 #2747: HBASE-24850 CellComparator perf improvement

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 18s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +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.  |
   ||| _ branch-2.3 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 23s |  branch-2.3 passed  |
   | +1 :green_heart: |  checkstyle  |   0m 52s |  branch-2.3 passed  |
   | +1 :green_heart: |  spotbugs  |   1m 46s |  branch-2.3 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 31s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 32s |  hbase-common: The patch generated 2 new + 192 unchanged - 1 fixed = 194 total (was 193)  |
   | -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  |  22m 31s |  Patch does not cause any errors with Hadoop 2.10.0 or 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   1m 48s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  48m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2747 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 27e8f81d2918 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 | branch-2.3 / 7b1e3e9268 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/1/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-2747/1/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-mapreduce U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2747/1/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=3.1.12 |
   | 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] saintstack commented on pull request #2747: HBASE-24850 CellComparator perf improvement

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


   Closing. You can read the comments here even if closed @ramkrish86 


----------------------------------------------------------------
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] ramkrish86 commented on a change in pull request #2747: HBASE-24850 CellComparator perf improvement

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



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
##########
@@ -292,4 +287,49 @@ public long heapSize() {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  @Override
+  public int getFamilyLengthPosition(int rowLen) {
+    return this.offset + Bytes.SIZEOF_SHORT + rowLen;
+  }
+
+  @Override
+  public int getFamilyInternalPosition(int familyLengthPosition) {
+    return familyLengthPosition + Bytes.SIZEOF_BYTE;
+  }
+
+  @Override
+  public int getQualifierInternalPosition(int famOffset, byte famLength) {
+    return famOffset + famLength;
+  }
+
+  private int getTimestampOffset(final int keylength) {
+    return this.offset + keylength - KeyValue.TIMESTAMP_TYPE_SIZE;
+  }
+
+  @Override
+  public long getTimestamp(int keyLength) {
+    int tsOffset = getTimestampOffset(keyLength);
+    return ByteBufferUtils.toLong(this.buf, tsOffset);
+  }
+
+  @Override
+  public byte getTypeByte(int keyLength) {
+    return ByteBufferUtils.toByte(this.buf, this.offset + keyLength - 1);
+  }
+
+  @Override
+  public int getKeyLength() {
+    return this.length;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return super.equals(other);
+  }
+
+  @Override
+  public int hashCode() {

Review comment:
       That is a spot bug inference. 




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