You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wa...@apache.org on 2023/05/10 02:15:18 UTC

[hbase] branch master updated: HBASE-27788 Skip family comparing when compare cells inner the store (#5171)

This is an automated email from the ASF dual-hosted git repository.

wangzheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 5d82d4f7466 HBASE-27788 Skip family comparing when compare cells inner the store (#5171)
5d82d4f7466 is described below

commit 5d82d4f7466f3c5983dde8784b369cd8b7ec927d
Author: bsglz <18...@qq.com>
AuthorDate: Wed May 10 10:15:07 2023 +0800

    HBASE-27788 Skip family comparing when compare cells inner the store (#5171)
    
    * HBASE-27788 Skip family comparing when compare cells inner the store
---
 .../apache/hadoop/hbase/CellComparatorImpl.java    | 51 +++++++++++---
 .../hadoop/hbase/InnerStoreCellComparator.java     | 82 ++++++++++++++++++++++
 .../apache/hadoop/hbase/io/hfile/HFileContext.java |  7 +-
 .../apache/hadoop/hbase/TestCellComparator.java    | 26 +++++++
 .../hadoop/hbase/io/hfile/FixedFileTrailer.java    | 18 +++--
 .../hadoop/hbase/regionserver/DefaultMemStore.java |  3 +-
 .../apache/hadoop/hbase/regionserver/HStore.java   | 12 +++-
 .../hadoop/hbase/regionserver/StoreFileReader.java |  7 +-
 .../hbase/io/hfile/TestFixedFileTrailer.java       |  9 +--
 9 files changed, 184 insertions(+), 31 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
index 2c19c0f1043..a94c3679f47 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
@@ -100,7 +100,7 @@ public class CellComparatorImpl implements CellComparator {
     return ignoreSequenceid ? diff : Long.compare(r.getSequenceId(), l.getSequenceId());
   }
 
-  private static int compareKeyValues(final KeyValue left, final KeyValue right) {
+  private int compareKeyValues(final KeyValue left, final KeyValue right) {
     int diff;
     // Compare Rows. Cache row length.
     int leftRowLength = left.getRowLength();
@@ -153,8 +153,8 @@ public class CellComparatorImpl implements CellComparator {
     // Compare families.
     int leftFamilyPosition = left.getFamilyOffset(leftFamilyLengthPosition);
     int rightFamilyPosition = right.getFamilyOffset(rightFamilyLengthPosition);
-    diff = Bytes.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
-      right.getFamilyArray(), rightFamilyPosition, rightFamilyLength);
+    diff = compareFamilies(left, leftFamilyPosition, leftFamilyLength, right, rightFamilyPosition,
+      rightFamilyLength);
     if (diff != 0) {
       return diff;
     }
@@ -183,7 +183,7 @@ public class CellComparatorImpl implements CellComparator {
     return (0xff & rightType) - (0xff & leftType);
   }
 
-  private static int compareBBKV(final ByteBufferKeyValue left, final ByteBufferKeyValue right) {
+  private int compareBBKV(final ByteBufferKeyValue left, final ByteBufferKeyValue right) {
     int diff;
     // Compare Rows. Cache row length.
     int leftRowLength = left.getRowLength();
@@ -236,8 +236,8 @@ public class CellComparatorImpl implements CellComparator {
     // Compare families.
     int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
     int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
-    diff = ByteBufferUtils.compareTo(left.getFamilyByteBuffer(), leftFamilyPosition,
-      leftFamilyLength, right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
+    diff = compareFamilies(left, leftFamilyPosition, leftFamilyLength, right, rightFamilyPosition,
+      rightFamilyLength);
     if (diff != 0) {
       return diff;
     }
@@ -265,7 +265,7 @@ public class CellComparatorImpl implements CellComparator {
     return (0xff & rightType) - (0xff & leftType);
   }
 
-  private static int compareKVVsBBKV(final KeyValue left, final ByteBufferKeyValue right) {
+  private int compareKVVsBBKV(final KeyValue left, final ByteBufferKeyValue right) {
     int diff;
     // Compare Rows. Cache row length.
     int leftRowLength = left.getRowLength();
@@ -318,8 +318,8 @@ public class CellComparatorImpl implements CellComparator {
     // Compare families.
     int leftFamilyPosition = left.getFamilyOffset(leftFamilyLengthPosition);
     int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
-    diff = ByteBufferUtils.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
-      right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
+    diff = compareFamilies(left, leftFamilyPosition, leftFamilyLength, right, rightFamilyPosition,
+      rightFamilyLength);
     if (diff != 0) {
       return diff;
     }
@@ -368,7 +368,10 @@ public class CellComparatorImpl implements CellComparator {
     return compareQualifiers(left, leftQualLen, right, rightQualLen);
   }
 
-  private int compareFamilies(Cell left, int leftFamLen, Cell right, int rightFamLen) {
+  /**
+   * This method will be overridden when we compare cells inner store to bypass family comparing.
+   */
+  protected int compareFamilies(Cell left, int leftFamLen, Cell right, int rightFamLen) {
     if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
         ((ByteBufferExtendedCell) left).getFamilyPosition(), leftFamLen,
@@ -448,6 +451,34 @@ public class CellComparatorImpl implements CellComparator {
       right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
   }
 
+  /**
+   * This method will be overridden when we compare cells inner store to bypass family comparing.
+   */
+  protected int compareFamilies(KeyValue left, int leftFamilyPosition, int leftFamilyLength,
+    KeyValue right, int rightFamilyPosition, int rightFamilyLength) {
+    return Bytes.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
+      right.getFamilyArray(), rightFamilyPosition, rightFamilyLength);
+  }
+
+  /**
+   * This method will be overridden when we compare cells inner store to bypass family comparing.
+   */
+  protected int compareFamilies(ByteBufferKeyValue left, int leftFamilyPosition,
+    int leftFamilyLength, ByteBufferKeyValue right, int rightFamilyPosition,
+    int rightFamilyLength) {
+    return ByteBufferUtils.compareTo(left.getFamilyByteBuffer(), leftFamilyPosition,
+      leftFamilyLength, right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
+  }
+
+  /**
+   * This method will be overridden when we compare cells inner store to bypass family comparing.
+   */
+  protected int compareFamilies(KeyValue left, int leftFamilyPosition, int leftFamilyLength,
+    ByteBufferKeyValue right, int rightFamilyPosition, int rightFamilyLength) {
+    return ByteBufferUtils.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
+      right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
+  }
+
   static int compareQualifiers(KeyValue left, KeyValue right) {
     // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
     // sharing gets us a few percent more throughput in compares. If changes here or there, make
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java
new file mode 100644
index 00000000000..7f6e87ebf91
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java
@@ -0,0 +1,82 @@
+/*
+ * 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.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * Compare two HBase cells inner store, skip compare family for better performance. Important!!! we
+ * should not make fake cell with fake family which length greater than zero inner store, otherwise
+ * this optimization cannot be used.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class InnerStoreCellComparator extends CellComparatorImpl {
+
+  private static final long serialVersionUID = 8186411895799094989L;
+
+  public static final InnerStoreCellComparator INNER_STORE_COMPARATOR =
+    new InnerStoreCellComparator();
+
+  @Override
+  protected int compareFamilies(Cell left, int leftFamilyLength, Cell right,
+    int rightFamilyLength) {
+    return leftFamilyLength - rightFamilyLength;
+  }
+
+  @Override
+  protected int compareFamilies(KeyValue left, int leftFamilyPosition, int leftFamilyLength,
+    KeyValue right, int rightFamilyPosition, int rightFamilyLength) {
+    return leftFamilyLength - rightFamilyLength;
+  }
+
+  @Override
+  protected int compareFamilies(ByteBufferKeyValue left, int leftFamilyPosition,
+    int leftFamilyLength, ByteBufferKeyValue right, int rightFamilyPosition,
+    int rightFamilyLength) {
+    return leftFamilyLength - rightFamilyLength;
+  }
+
+  @Override
+  protected int compareFamilies(KeyValue left, int leftFamilyPosition, int leftFamilyLength,
+    ByteBufferKeyValue right, int rightFamilyPosition, int rightFamilyLength) {
+    return leftFamilyLength - rightFamilyLength;
+  }
+
+  /**
+   * Utility method that makes a guess at comparator to use based off passed tableName. Use in
+   * extreme when no comparator specified.
+   * @return CellComparator to use going off the {@code tableName} passed.
+   */
+  public static CellComparator getInnerStoreCellComparator(TableName tableName) {
+    return getInnerStoreCellComparator(tableName.toBytes());
+  }
+
+  /**
+   * Utility method that makes a guess at comparator to use based off passed tableName. Use in
+   * extreme when no comparator specified.
+   * @return CellComparator to use going off the {@code tableName} passed.
+   */
+  public static CellComparator getInnerStoreCellComparator(byte[] tableName) {
+    return Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes())
+      ? MetaCellComparator.META_COMPARATOR
+      : InnerStoreCellComparator.INNER_STORE_COMPARATOR;
+  }
+}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java
index b371cf83867..98520d949af 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hbase.io.hfile;
 
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.InnerStoreCellComparator;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
@@ -121,8 +121,9 @@ public class HFileContext implements HeapSize, Cloneable {
     // If no cellComparator specified, make a guess based off tablename. If hbase:meta, then should
     // be the meta table comparator. Comparators are per table.
     this.cellComparator = cellComparator != null ? cellComparator
-      : this.tableName != null ? CellComparatorImpl.getCellComparator(this.tableName)
-      : CellComparator.getInstance();
+      : this.tableName != null
+        ? InnerStoreCellComparator.getInnerStoreCellComparator(this.tableName)
+      : InnerStoreCellComparator.INNER_STORE_COMPARATOR;
   }
 
   /** Returns true when on-disk blocks are compressed, and/or encrypted; false otherwise. */
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
index f4da9bf2f7b..c4b24728b42 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
@@ -40,10 +40,13 @@ public class TestCellComparator {
     HBaseClassTestRule.forClass(TestCellComparator.class);
 
   private CellComparator comparator = CellComparator.getInstance();
+  private CellComparator innerStoreComparator = InnerStoreCellComparator.INNER_STORE_COMPARATOR;
+
   byte[] row1 = Bytes.toBytes("row1");
   byte[] row2 = Bytes.toBytes("row2");
   byte[] row_1_0 = Bytes.toBytes("row10");
 
+  byte[] fam0 = HConstants.EMPTY_BYTE_ARRAY;
   byte[] fam1 = Bytes.toBytes("fam1");
   byte[] fam2 = Bytes.toBytes("fam2");
   byte[] fam_1_2 = Bytes.toBytes("fam12");
@@ -76,6 +79,29 @@ public class TestCellComparator {
     assertTrue(CellUtil.equals(kv1, kv2));
   }
 
+  @Test
+  public void testCompareCellsWithEmptyFamily() {
+    KeyValue kv1 = new KeyValue(row1, fam0, qual1, val);
+    KeyValue kv2 = new KeyValue(row1, fam1, qual1, val);
+    assertTrue(comparator.compare(kv1, kv2) < 0);
+    assertTrue(innerStoreComparator.compare(kv1, kv2) < 0);
+
+    kv1 = new KeyValue(row1, fam0, qual2, val);
+    kv2 = new KeyValue(row1, fam0, qual1, val);
+    assertTrue(comparator.compare(kv1, kv2) > 0);
+    assertTrue(innerStoreComparator.compare(kv1, kv2) > 0);
+
+    kv1 = new KeyValue(row1, fam0, qual2, val);
+    kv2 = new KeyValue(row1, fam0, qual1, val);
+    assertTrue(comparator.compareFamilies(kv1, kv2) == 0);
+    assertTrue(innerStoreComparator.compareFamilies(kv1, kv2) == 0);
+
+    kv1 = new KeyValue(row1, fam1, qual2, val);
+    kv2 = new KeyValue(row1, fam1, qual1, val);
+    assertTrue(comparator.compareFamilies(kv1, kv2) == 0);
+    assertTrue(innerStoreComparator.compareFamilies(kv1, kv2) == 0);
+  }
+
   @Test
   public void testCompareCellWithKey() throws Exception {
     KeyValue kv1 = new KeyValue(row1, fam1, qual1, val);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
index 134485135ed..2a405197a48 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
@@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellComparatorImpl;
+import org.apache.hadoop.hbase.InnerStoreCellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MetaCellComparator;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -121,7 +122,8 @@ public class FixedFileTrailer {
    * Raw key comparator class name in version 3
    */
   // We could write the actual class name from 2.0 onwards and handle BC
-  private String comparatorClassName = CellComparator.getInstance().getClass().getName();
+  private String comparatorClassName =
+    InnerStoreCellComparator.INNER_STORE_COMPARATOR.getClass().getName();
 
   /**
    * The encryption key
@@ -574,7 +576,10 @@ public class FixedFileTrailer {
    */
   @Deprecated
   private String getHBase1CompatibleName(final String comparator) {
-    if (comparator.equals(CellComparatorImpl.class.getName())) {
+    if (
+      comparator.equals(CellComparatorImpl.class.getName())
+        || comparator.equals(InnerStoreCellComparator.class.getName())
+    ) {
       return KeyValue.COMPARATOR.getClass().getName();
     }
     if (comparator.equals(MetaCellComparator.class.getName())) {
@@ -593,7 +598,7 @@ public class FixedFileTrailer {
         || comparatorClassName.equals(KeyValue.COMPARATOR.getClass().getName())
         || (comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator"))
     ) {
-      comparatorKlass = CellComparatorImpl.class;
+      comparatorKlass = InnerStoreCellComparator.class;
     } else if (
       comparatorClassName.equals(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName())
         || comparatorClassName.equals(KeyValue.META_COMPARATOR.getClass().getName())
@@ -622,8 +627,11 @@ public class FixedFileTrailer {
   }
 
   static CellComparator createComparator(String comparatorClassName) throws IOException {
-    if (comparatorClassName.equals(CellComparatorImpl.COMPARATOR.getClass().getName())) {
-      return CellComparatorImpl.COMPARATOR;
+    if (
+      comparatorClassName
+        .equals(InnerStoreCellComparator.INNER_STORE_COMPARATOR.getClass().getName())
+    ) {
+      return InnerStoreCellComparator.INNER_STORE_COMPARATOR;
     } else
       if (comparatorClassName.equals(MetaCellComparator.META_COMPARATOR.getClass().getName())) {
         return MetaCellComparator.META_COMPARATOR;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index 12ec4a80fec..420dad51e37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.InnerStoreCellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -58,7 +59,7 @@ public class DefaultMemStore extends AbstractMemStore {
    * Default constructor. Used for tests.
    */
   public DefaultMemStore() {
-    this(HBaseConfiguration.create(), CellComparator.getInstance(), null);
+    this(HBaseConfiguration.create(), InnerStoreCellComparator.INNER_STORE_COMPARATOR, null);
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 5b52ab07a48..5e2bf00f85b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -61,7 +61,9 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.InnerStoreCellComparator;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
+import org.apache.hadoop.hbase.MetaCellComparator;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.FailedArchiveException;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -276,7 +278,8 @@ public class HStore
     long ttl = determineTTLFromFamily(family);
     // Why not just pass a HColumnDescriptor in here altogether? Even if have
     // to clone it?
-    scanInfo = new ScanInfo(conf, family, ttl, timeToPurgeDeletes, region.getCellComparator());
+    scanInfo =
+      new ScanInfo(conf, family, ttl, timeToPurgeDeletes, this.storeContext.getComparator());
     this.memstore = getMemstore();
 
     this.offPeakHours = OffPeakHours.getInstance(conf);
@@ -329,8 +332,11 @@ public class HStore
     return new StoreContext.Builder().withBlockSize(family.getBlocksize())
       .withEncryptionContext(EncryptionUtil.createEncryptionContext(conf, family))
       .withBloomType(family.getBloomFilterType()).withCacheConfig(createCacheConf(family))
-      .withCellComparator(region.getCellComparator()).withColumnFamilyDescriptor(family)
-      .withCompactedFilesSupplier(this::getCompactedFiles)
+      .withCellComparator(region.getTableDescriptor().isMetaTable() || conf
+        .getBoolean(HRegion.USE_META_CELL_COMPARATOR, HRegion.DEFAULT_USE_META_CELL_COMPARATOR)
+          ? MetaCellComparator.META_COMPARATOR
+          : InnerStoreCellComparator.INNER_STORE_COMPARATOR)
+      .withColumnFamilyDescriptor(family).withCompactedFilesSupplier(this::getCompactedFiles)
       .withRegionFileSystem(region.getRegionFileSystem())
       .withFavoredNodesSupplier(this::getFavoredNodes)
       .withFamilyStoreDirectoryPath(
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
index a2778e54a72..dc8b06200ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -401,7 +401,7 @@ public class StoreFileReader {
         // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
         if (keyIsAfterLast) {
           if (bloomFilterType == BloomType.ROWCOL) {
-            keyIsAfterLast = (CellComparator.getInstance().compare(kvKey, lastBloomKeyOnlyKV)) > 0;
+            keyIsAfterLast = (getComparator().compare(kvKey, lastBloomKeyOnlyKV)) > 0;
           } else {
             keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0);
           }
@@ -415,10 +415,7 @@ public class StoreFileReader {
           Cell rowBloomKey = PrivateCellUtil.createFirstOnRow(kvKey);
           // hbase:meta does not have blooms. So we need not have special interpretation
           // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
-          if (
-            keyIsAfterLast
-              && (CellComparator.getInstance().compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0
-          ) {
+          if (keyIsAfterLast && (getComparator().compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) {
             exists = false;
           } else {
             exists = bloomFilter.contains(kvKey, bloom, BloomType.ROWCOL)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
index 38b31e0fce0..442f62e505d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.InnerStoreCellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MetaCellComparator;
 import org.apache.hadoop.hbase.testclassification.IOTests;
@@ -119,11 +120,11 @@ public class TestFixedFileTrailer {
   public void testCreateComparator() throws IOException {
     FixedFileTrailer t = new FixedFileTrailer(version, HFileReaderImpl.PBUF_TRAILER_MINOR_VERSION);
     try {
-      assertEquals(CellComparatorImpl.class,
+      assertEquals(InnerStoreCellComparator.class,
         t.createComparator(KeyValue.COMPARATOR.getLegacyKeyComparatorName()).getClass());
-      assertEquals(CellComparatorImpl.class,
+      assertEquals(InnerStoreCellComparator.class,
         t.createComparator(KeyValue.COMPARATOR.getClass().getName()).getClass());
-      assertEquals(CellComparatorImpl.class,
+      assertEquals(InnerStoreCellComparator.class,
         t.createComparator(CellComparator.class.getName()).getClass());
       assertEquals(MetaCellComparator.class,
         t.createComparator(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName()).getClass());
@@ -161,7 +162,7 @@ public class TestFixedFileTrailer {
 
     t.setLastDataBlockOffset(291);
     t.setNumDataIndexLevels(3);
-    t.setComparatorClass(CellComparatorImpl.COMPARATOR.getClass());
+    t.setComparatorClass(InnerStoreCellComparator.INNER_STORE_COMPARATOR.getClass());
     t.setFirstDataBlockOffset(9081723123L); // Completely unrealistic.
     t.setUncompressedDataIndexSize(827398717L); // Something random.