From 872f13ae52f0105a7a27a9844d5974b8ace293ad Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Tue, 19 Aug 2025 11:32:51 +0900 Subject: [PATCH 01/21] Add row-level cache for the get operation --- .../hadoop/hbase/client/TableDescriptor.java | 7 + .../hbase/client/TableDescriptorBuilder.java | 23 + .../org/apache/hadoop/hbase/HConstants.java | 8 + .../io/encoding/BufferedDataBlockEncoder.java | 4 +- .../hadoop/hbase/io/hfile/BlockType.java | 9 + .../MetricsRegionServerSource.java | 2 + .../MetricsRegionServerSourceImpl.java | 2 + .../MetricsRegionServerWrapper.java | 4 + .../hadoop/hbase/io/hfile/CacheStats.java | 16 + .../hadoop/hbase/io/hfile/RowCacheKey.java | 72 +++ .../hadoop/hbase/regionserver/HRegion.java | 18 + .../hbase/regionserver/HRegionServer.java | 3 + .../MetricsRegionServerWrapperImpl.java | 10 + .../hbase/regionserver/RSRpcServices.java | 84 +-- .../hbase/regionserver/RowCacheService.java | 361 +++++++++++++ .../hadoop/hbase/regionserver/RowCells.java | 80 +++ .../io/hfile/TestCombinedBlockCache.java | 6 + .../hbase/io/hfile/TestRowCacheKey.java | 114 +++++ .../MetricsRegionServerWrapperStub.java | 10 + .../regionserver/TestMetricsRegionServer.java | 2 + .../hbase/regionserver/TestRowCache.java | 481 ++++++++++++++++++ .../regionserver/TestRowCacheCanCacheRow.java | 216 ++++++++ .../TestRowCacheClearRegionBlockCache.java | 122 +++++ .../TestRowCacheConfigurationObserver.java | 204 ++++++++ .../regionserver/TestRowCacheHRegion.java | 97 ++++ .../regionserver/TestRowCacheService.java | 263 ++++++++++ .../hbase/regionserver/TestRowCells.java | 74 +++ .../tool/TestBulkLoadHFilesRowCache.java | 160 ++++++ 28 files changed, 2418 insertions(+), 34 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/RowCacheKey.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowCacheKey.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheHRegion.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCells.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java index 817f9e2d4b1b..6d7786b5c88a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java @@ -316,4 +316,11 @@ default boolean matchReplicationScope(boolean enabled) { } return !enabled; } + + /** + * Checks whether row caching is enabled for this table. Note that row caching applies only at the + * entire row level, not at the column family level. + * @return {@code true} if row cache is enabled, otherwise {@code false} + */ + boolean isRowCacheEnabled(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index 8636b006e83d..64bd686b3360 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -227,6 +227,15 @@ public class TableDescriptorBuilder { private final static Map DEFAULT_VALUES = new HashMap<>(); private final static Set RESERVED_KEYWORDS = new HashSet<>(); + /** + * Used by HBase Shell interface to access this metadata attribute which denotes if the row cache + * is enabled. + */ + @InterfaceAudience.Private + public static final String ROW_CACHE_ENABLED = "ROW_CACHE_ENABLED"; + private static final Bytes ROW_CACHE_ENABLED_KEY = new Bytes(Bytes.toBytes(ROW_CACHE_ENABLED)); + private static final boolean DEFAULT_ROW_CACHE_ENABLED = false; + static { DEFAULT_VALUES.put(MAX_FILESIZE, String.valueOf(HConstants.DEFAULT_MAX_FILE_SIZE)); DEFAULT_VALUES.put(READONLY, String.valueOf(DEFAULT_READONLY)); @@ -565,6 +574,11 @@ public TableDescriptor build() { return new ModifyableTableDescriptor(desc); } + public TableDescriptorBuilder setRowCacheEnabled(boolean rowCacheEnabled) { + desc.setRowCacheEnabled(rowCacheEnabled); + return this; + } + private static final class ModifyableTableDescriptor implements TableDescriptor, Comparable { @@ -1510,6 +1524,15 @@ public Optional getRegionServerGroup() { return Optional.empty(); } } + + @Override + public boolean isRowCacheEnabled() { + return getOrDefault(ROW_CACHE_ENABLED_KEY, Boolean::valueOf, DEFAULT_ROW_CACHE_ENABLED); + } + + public ModifyableTableDescriptor setRowCacheEnabled(boolean enabled) { + return setValue(ROW_CACHE_ENABLED_KEY, Boolean.toString(enabled)); + } } /** diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 1051686d32e8..70241f6404db 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1033,6 +1033,14 @@ public enum OperationStatusCode { public static final long HBASE_CLIENT_SCANNER_ONHEAP_BLOCK_CACHE_FIXED_SIZE_DEFAULT = 32 * 1024 * 1024L; + /** + * Configuration key for the minimum number of HFiles required to activate the Row Cache. If the + * number of HFiles is less than this value, the Row Cache does not operate even if it is enabled + * at the table level. + */ + public static final String ROW_CACHE_ACTIVATE_MIN_HFILES_KEY = "row.cache.activate.min.hfiles"; + public static final int ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT = 2; + /** * Configuration key for setting pread must read both necessaryLen and extraLen, default is * disabled. This is an optimized flag for reading HFile from blob storage. diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java index 5ec39fa5803d..c3d113c8b655 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java @@ -547,8 +547,8 @@ public void setTimestamp(byte[] ts) throws IOException { @Override public ExtendedCell deepClone() { - // This is not used in actual flow. Throwing UnsupportedOperationException - throw new UnsupportedOperationException(); + // To garbage collect the objects referenced by this cell, we need to deep clone it + return ExtendedCell.super.deepClone(); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java index dd1090833226..e73620e78bb5 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java @@ -80,6 +80,14 @@ public int getId() { /** Fixed file trailer, both versions (always just a magic string) */ TRAILER("TRABLK\"$", BlockCategory.META), + // Pseudo block + + /** + * Cells of a row for row cache. This is a pseudo block type. It only exists to share the + * BlockCache interface. + */ + ROW_CELLS("ROWCELLS", BlockCategory.ROW), + // Legacy blocks /** Block index magic string in version 1 */ @@ -91,6 +99,7 @@ public enum BlockCategory { INDEX, BLOOM, ALL_CATEGORIES, + ROW, UNKNOWN; /** diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java index c88a77b51407..ad11e3143c91 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java @@ -386,6 +386,7 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo String BLOCK_CACHE_GENERAL_BLOOM_META_MISS_COUNT = "blockCacheGeneralBloomMetaMissCount"; String BLOCK_CACHE_DELETE_FAMILY_BLOOM_MISS_COUNT = "blockCacheDeleteFamilyBloomMissCount"; String BLOCK_CACHE_TRAILER_MISS_COUNT = "blockCacheTrailerMissCount"; + String BLOCK_CACHE_ROW_MISS_COUNT = "blockCacheRowMissCount"; String BLOCK_CACHE_DATA_HIT_COUNT = "blockCacheDataHitCount"; String BLOCK_CACHE_ENCODED_DATA_HIT_COUNT = "blockCacheEncodedDataHitCount"; String BLOCK_CACHE_LEAF_INDEX_HIT_COUNT = "blockCacheLeafIndexHitCount"; @@ -397,6 +398,7 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo String BLOCK_CACHE_GENERAL_BLOOM_META_HIT_COUNT = "blockCacheGeneralBloomMetaHitCount"; String BLOCK_CACHE_DELETE_FAMILY_BLOOM_HIT_COUNT = "blockCacheDeleteFamilyBloomHitCount"; String BLOCK_CACHE_TRAILER_HIT_COUNT = "blockCacheTrailerHitCount"; + String BLOCK_CACHE_ROW_HIT_COUNT = "blockCacheRowHitCount"; String L1_CACHE_FREE_SIZE = "l1CacheFreeSize"; String L1_CACHE_FREE_SIZE_DESC = "Amount of free bytes in the L1 cache"; String L1_CACHE_SIZE = "l1CacheSize"; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java index b214c8f8f4e7..8fecb3c6f66d 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java @@ -436,6 +436,7 @@ public void getMetrics(MetricsCollector metricsCollector, boolean all) { .addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_MISS_COUNT, ""), rsWrap.getDeleteFamilyBloomMissCount()) .addCounter(Interns.info(BLOCK_CACHE_TRAILER_MISS_COUNT, ""), rsWrap.getTrailerMissCount()) + .addCounter(Interns.info(BLOCK_CACHE_ROW_MISS_COUNT, ""), rsWrap.getRowMissCount()) .addCounter(Interns.info(BLOCK_CACHE_DATA_HIT_COUNT, ""), rsWrap.getDataHitCount()) .addCounter(Interns.info(BLOCK_CACHE_LEAF_INDEX_HIT_COUNT, ""), rsWrap.getLeafIndexHitCount()) @@ -452,6 +453,7 @@ public void getMetrics(MetricsCollector metricsCollector, boolean all) { .addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_HIT_COUNT, ""), rsWrap.getDeleteFamilyBloomHitCount()) .addCounter(Interns.info(BLOCK_CACHE_TRAILER_HIT_COUNT, ""), rsWrap.getTrailerHitCount()) + .addCounter(Interns.info(BLOCK_CACHE_ROW_HIT_COUNT, ""), rsWrap.getRowHitCount()) .addCounter(Interns.info(UPDATES_BLOCKED_TIME, UPDATES_BLOCKED_DESC), rsWrap.getUpdatesBlockedTime()) .addCounter(Interns.info(FLUSHED_CELLS, FLUSHED_CELLS_DESC), rsWrap.getFlushedCellsCount()) diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java index 5b957d9bf08f..a665aec9b89f 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java @@ -615,6 +615,8 @@ public interface MetricsRegionServerWrapper { long getTrailerMissCount(); + long getRowMissCount(); + long getDataHitCount(); long getLeafIndexHitCount(); @@ -635,6 +637,8 @@ public interface MetricsRegionServerWrapper { long getTrailerHitCount(); + long getRowHitCount(); + long getTotalRowActionRequestCount(); long getByteBuffAllocatorHeapAllocationBytes(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java index fbf3e1087019..9875b0c23362 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java @@ -91,6 +91,7 @@ public class CacheStats { private final LongAdder generalBloomMetaMissCount = new LongAdder(); private final LongAdder deleteFamilyBloomMissCount = new LongAdder(); private final LongAdder trailerMissCount = new LongAdder(); + private final LongAdder rowMissCount = new LongAdder(); private final LongAdder dataHitCount = new LongAdder(); private final LongAdder leafIndexHitCount = new LongAdder(); @@ -102,6 +103,7 @@ public class CacheStats { private final LongAdder generalBloomMetaHitCount = new LongAdder(); private final LongAdder deleteFamilyBloomHitCount = new LongAdder(); private final LongAdder trailerHitCount = new LongAdder(); + private final LongAdder rowHitCount = new LongAdder(); // Executor for periodic cache stats rolling private ScheduledExecutorService metricsRollerScheduler; @@ -219,6 +221,9 @@ public void miss(boolean caching, boolean primary, BlockType type) { case TRAILER: trailerMissCount.increment(); break; + case ROW_CELLS: + rowMissCount.increment(); + break; default: // If there's a new type that's fine // Ignore it for now. This is metrics don't exception. @@ -266,6 +271,9 @@ public void hit(boolean caching, boolean primary, BlockType type) { case TRAILER: trailerHitCount.increment(); break; + case ROW_CELLS: + rowHitCount.increment(); + break; default: // If there's a new type that's fine // Ignore it for now. This is metrics don't exception. @@ -376,6 +384,14 @@ public long getTrailerHitCount() { return trailerHitCount.sum(); } + public long getRowHitCount() { + return rowHitCount.sum(); + } + + public long getRowMissCount() { + return rowMissCount.sum(); + } + public long getRequestCount() { return getHitCount() + getMissCount(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/RowCacheKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/RowCacheKey.java new file mode 100644 index 000000000000..d26af3213c1f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/RowCacheKey.java @@ -0,0 +1,72 @@ +/* + * 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.hfile; + +import java.util.Arrays; +import java.util.Objects; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Cache Key for use with implementations of {@link BlockCache} + */ +@InterfaceAudience.Private +public class RowCacheKey extends BlockCacheKey { + private static final long serialVersionUID = -686874540957524887L; + public static final long FIXED_OVERHEAD = ClassSize.estimateBase(RowCacheKey.class, false); + + private final byte[] rowKey; + // Row cache keys should not be evicted on close, since the cache may contain many entries and + // eviction would be slow. Instead, the region’s rowCacheSeqNum is used to generate new keys that + // ignore the existing cache when the region is reopened or bulk-loaded. + private final long rowCacheSeqNum; + + public RowCacheKey(HRegion region, byte[] rowKey) { + super(region.getRegionInfo().getEncodedName(), 0, region.getRegionInfo().getReplicaId() == 0, + BlockType.ROW_CELLS); + + this.rowKey = Objects.requireNonNull(rowKey, "rowKey cannot be null"); + this.rowCacheSeqNum = region.getRowCacheSeqNum(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + RowCacheKey that = (RowCacheKey) o; + return rowCacheSeqNum == that.rowCacheSeqNum && Arrays.equals(rowKey, that.rowKey); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), Arrays.hashCode(rowKey), Long.hashCode(rowCacheSeqNum)); + } + + @Override + public String toString() { + return super.toString() + '_' + Bytes.toStringBinary(this.rowKey) + '_' + rowCacheSeqNum; + } + + @Override + public long heapSize() { + return FIXED_OVERHEAD + ClassSize.align(rowKey.length); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 7936197ff8d8..7ef3e67a25a3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -65,6 +65,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; @@ -433,6 +434,11 @@ public MetricsTableRequests getMetricsTableRequests() { */ private long openSeqNum = HConstants.NO_SEQNUM; + /** + * Basically the same as openSeqNum, but it is updated when bulk load is done. + */ + private final AtomicLong rowCacheSeqNum = new AtomicLong(HConstants.NO_SEQNUM); + /** * The default setting for whether to enable on-demand CF loading for scan requests to this * region. Requests can override it. @@ -7868,6 +7874,7 @@ private HRegion openHRegion(final CancelableProgressable reporter) throws IOExce LOG.debug("checking classloading for " + this.getRegionInfo().getEncodedName()); TableDescriptorChecker.checkClassLoading(cConfig, htableDescriptor); this.openSeqNum = initialize(reporter); + this.rowCacheSeqNum.set(this.openSeqNum); this.mvcc.advanceTo(openSeqNum); // The openSeqNum must be increased every time when a region is assigned, as we rely on it to // determine whether a region has been successfully reopened. So here we always write open @@ -8696,6 +8703,17 @@ public long getOpenSeqNum() { return this.openSeqNum; } + public long getRowCacheSeqNum() { + return this.rowCacheSeqNum.get(); + } + + /** + * This is used to invalidate the entire row cache after bulk loading. + */ + public void increaseRowCacheSeqNum() { + this.rowCacheSeqNum.incrementAndGet(); + } + @Override public Map getMaxStoreSeqId() { return this.maxSeqIdInStores; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 350baca36f46..6ab78b14fbab 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -3408,6 +3408,9 @@ CacheEvictionStats clearRegionBlockCache(Region region) { } } + // evict the entire row cache + evictedBlocks += blockCache.evictBlocksByHfileName(region.getRegionInfo().getEncodedName()); + return CacheEvictionStats.builder().withEvictedBlocks(evictedBlocks).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index c8f7f96a033b..3732d492794e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -1144,6 +1144,11 @@ public long getTrailerMissCount() { return this.cacheStats != null ? this.cacheStats.getTrailerMissCount() : 0L; } + @Override + public long getRowMissCount() { + return this.cacheStats != null ? this.cacheStats.getRowMissCount() : 0L; + } + @Override public long getDataHitCount() { return this.cacheStats != null ? this.cacheStats.getDataHitCount() : 0L; @@ -1194,6 +1199,11 @@ public long getTrailerHitCount() { return this.cacheStats != null ? this.cacheStats.getTrailerHitCount() : 0L; } + @Override + public long getRowHitCount() { + return this.cacheStats != null ? this.cacheStats.getRowHitCount() : 0L; + } + @Override public long getByteBuffAllocatorHeapAllocationBytes() { return ByteBuffAllocator.getHeapAllocationBytes(allocator, ByteBuffAllocator.HEAP); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index bd232addcec5..0a84afe046f1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -353,6 +353,11 @@ public class RSRpcServices extends HBaseRpcServicesBase public static final String REGIONSERVER_BOOTSTRAP_NODES_SERVICE_CONFIG = "hbase.regionserver.bootstrap.nodes.executorService"; + /** + * The row cache service + */ + private final RowCacheService rowCacheService = new RowCacheService(getConfiguration()); + /** * An Rpc callback for closing a RegionScanner. */ @@ -667,7 +672,8 @@ private CheckAndMutateResult checkAndMutate(HRegion region, List Row.COMPARATOR.compare(v1, v2)); } - OperationStatus[] codes = region.batchMutate(mArray, atomic, nonceGroup, nonce); + OperationStatus[] codes = + rowCacheService.batchMutate(region, mArray, atomic, nonceGroup, nonce); // When atomic is true, it indicates that the mutateRow API or the batch API with // RowMutations is called. In this case, we need to merge the results of the @@ -2315,6 +2322,11 @@ public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller, @Override public BulkLoadHFileResponse bulkLoadHFile(final RpcController controller, final BulkLoadHFileRequest request) throws ServiceException { + return rowCacheService.bulkLoadHFile(this, request); + } + + BulkLoadHFileResponse bulkLoadHFileInternal(final BulkLoadHFileRequest request) + throws ServiceException { long start = EnvironmentEdgeManager.currentTime(); List clusterIds = new ArrayList<>(request.getClusterIdsList()); if (clusterIds.contains(this.server.getClusterId())) { @@ -2571,8 +2583,7 @@ private Result get(Get get, HRegion region, RegionScannersCloseCallBack closeCal RegionScannerImpl scanner = null; long blockBytesScannedBefore = context.getBlockBytesScanned(); try { - scanner = region.getScanner(scan); - scanner.next(results); + scanner = rowCacheService.getScanner(region, get, scan, results); } finally { if (scanner != null) { if (closeCallBack == null) { @@ -2981,33 +2992,10 @@ public MutateResponse mutate(final RpcController rpcc, final MutateRequest reque builder.setMetrics(ProtobufUtil.toQueryMetrics(result.getMetrics())); } } else { - Result r = null; - Boolean processed = null; - MutationType type = mutation.getMutateType(); - switch (type) { - case APPEND: - // TODO: this doesn't actually check anything. - r = append(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement, - context); - break; - case INCREMENT: - // TODO: this doesn't actually check anything. - r = increment(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement, - context); - break; - case PUT: - put(region, quota, mutation, cellScanner, spaceQuotaEnforcement); - processed = Boolean.TRUE; - break; - case DELETE: - delete(region, quota, mutation, cellScanner, spaceQuotaEnforcement); - processed = Boolean.TRUE; - break; - default: - throw new DoNotRetryIOException("Unsupported mutate type: " + type.name()); - } - if (processed != null) { - builder.setProcessed(processed); + Result r = rowCacheService.mutate(this, region, mutation, quota, cellScanner, nonceGroup, + spaceQuotaEnforcement, context); + if (r == Result.EMPTY_RESULT) { + builder.setProcessed(true); } boolean clientCellBlockSupported = isClientCellBlockSupport(context); addResult(builder, r, controller, clientCellBlockSupported); @@ -3026,10 +3014,41 @@ public MutateResponse mutate(final RpcController rpcc, final MutateRequest reque } } + Result mutateInternal(MutationProto mutation, HRegion region, OperationQuota quota, + CellScanner cellScanner, long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement, + RpcCallContext context) throws IOException { + MutationType type = mutation.getMutateType(); + return switch (type) { + case APPEND -> + // TODO: this doesn't actually check anything. + append(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement, context); + case INCREMENT -> + // TODO: this doesn't actually check anything. + increment(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement, + context); + case PUT -> { + put(region, quota, mutation, cellScanner, spaceQuotaEnforcement); + yield Result.EMPTY_RESULT; + } + case DELETE -> { + delete(region, quota, mutation, cellScanner, spaceQuotaEnforcement); + yield Result.EMPTY_RESULT; + } + }; + } + private void put(HRegion region, OperationQuota quota, MutationProto mutation, CellScanner cellScanner, ActivePolicyEnforcement spaceQuota) throws IOException { long before = EnvironmentEdgeManager.currentTime(); Put put = ProtobufUtil.toPut(mutation, cellScanner); + // Put with TTL is not allowed on tables with row cache enabled, because cached rows cannot + // track TTL expiration + if (region.getTableDescriptor().isRowCacheEnabled()) { + if (put.getTTL() != Long.MAX_VALUE) { + throw new DoNotRetryIOException( + "Tables with row cache enabled do not allow setting TTL on Puts"); + } + } checkCellSizeLimit(region, put); spaceQuota.getPolicyEnforcement(region).check(put); quota.addMutation(put); @@ -3074,7 +3093,7 @@ private CheckAndMutateResult checkAndMutate(HRegion region, OperationQuota quota result = region.getCoprocessorHost().preCheckAndMutate(checkAndMutate); } if (result == null) { - result = region.checkAndMutate(checkAndMutate, nonceGroup, nonce); + result = rowCacheService.checkAndMutate(region, checkAndMutate, nonceGroup, nonce); if (region.getCoprocessorHost() != null) { result = region.getCoprocessorHost().postCheckAndMutate(checkAndMutate, result); } @@ -4023,6 +4042,7 @@ private void setReloadableGuardrails(Configuration conf) { public void onConfigurationChange(Configuration conf) { super.onConfigurationChange(conf); setReloadableGuardrails(conf); + rowCacheService.updateConf(conf); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java new file mode 100644 index 000000000000..636961ae5f1b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -0,0 +1,361 @@ +/* + * 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.regionserver; + +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.CheckAndMutate; +import org.apache.hadoop.hbase.client.CheckAndMutateResult; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Consistency; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.hfile.RowCacheKey; +import org.apache.hadoop.hbase.ipc.RpcCallContext; +import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement; +import org.apache.hadoop.hbase.quotas.OperationQuota; + +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse; + +/** + * It is responsible for populating the row cache and retrieving rows from it. + */ +class RowCacheService { + /** + * A barrier that prevents the row cache from being populated during table operations, such as + * bulk loads. It is implemented as a counter to address issues that arise when the same table is + * updated concurrently. + */ + private final Map tableLevelBarrierMap = new ConcurrentHashMap<>(); + /** + * A barrier that prevents the row cache from being populated during row mutations. It is + * implemented as a counter to address issues that arise when the same row is mutated + * concurrently. + */ + private final Map rowLevelBarrierMap = new ConcurrentHashMap<>(); + private int activateMinHFiles; + + @FunctionalInterface + interface RowOperation { + R execute() throws IOException; + } + + RowCacheService(Configuration conf) { + updateConf(conf); + } + + synchronized void updateConf(Configuration conf) { + this.activateMinHFiles = + conf.getInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT); + } + + RegionScannerImpl getScanner(HRegion region, Get get, Scan scan, List results) + throws IOException { + if (!canCacheRow(get, region)) { + return getScannerInternal(region, scan, results); + } + + RowCacheKey key = new RowCacheKey(region, get.getRow()); + + // Try get from row cache + if (tryGetFromCache(region, key, get, results)) { + // Cache is hit, and then no scanner is created + return null; + } + + RegionScannerImpl scanner = getScannerInternal(region, scan, results); + + // The row cache is ineffective when the number of store files is small. If the number + // of store files falls below the minimum threshold, rows will not be cached + if (hasSufficientHFiles(region)) { + populateCache(region, results, key); + } + + return scanner; + } + + private RegionScannerImpl getScannerInternal(HRegion region, Scan scan, List results) + throws IOException { + RegionScannerImpl scanner = region.getScanner(scan); + scanner.next(results); + return scanner; + } + + private boolean tryGetFromCache(HRegion region, RowCacheKey key, Get get, List results) { + RowCells row = + (RowCells) region.getBlockCache().getBlock(key, get.getCacheBlocks(), false, true); + + if (row == null) { + return false; + } + + results.addAll(row.getCells()); + region.addReadRequestsCount(1); + if (region.getMetrics() != null) { + region.getMetrics().updateReadRequestCount(); + } + return true; + } + + private boolean hasSufficientHFiles(HRegion region) { + return region.getStores().stream() + .anyMatch(store -> store.getStorefilesCount() >= activateMinHFiles); + } + + private void populateCache(HRegion region, List results, RowCacheKey key) { + // The row cache is populated only when no table level barriers remain + tableLevelBarrierMap.computeIfAbsent(region.getRegionInfo().getTable(), t -> { + // The row cache is populated only when no row level barriers remain + rowLevelBarrierMap.computeIfAbsent(key, k -> { + try { + region.getBlockCache().cacheBlock(key, new RowCells(results), false); + } catch (CloneNotSupportedException ignored) { + // Not able to cache row cells, ignore + } + return null; + }); + return null; + }); + } + + BulkLoadHFileResponse bulkLoadHFile(RSRpcServices rsRpcServices, BulkLoadHFileRequest request) + throws ServiceException { + HRegion region; + try { + region = rsRpcServices.getRegion(request.getRegion()); + } catch (IOException ie) { + throw new ServiceException(ie); + } + + if (!region.getTableDescriptor().isRowCacheEnabled()) { + return bulkLoad(rsRpcServices, request); + } + + // Since bulkload modifies the store files, the row cache should be disabled until the bulkload + // is finished. + createTableLevelBarrier(region.getRegionInfo().getTable()); + try { + // We do not invalidate the entire row cache directly, as it contains a large number of + // entries and takes a long time. Instead, we increment rowCacheSeqNum, which is used when + // constructing a RowCacheKey, thereby making the existing row cache entries stale. + increaseRowCacheSeqNum(region); + return bulkLoad(rsRpcServices, request); + } finally { + // The row cache for the table has been enabled again + removeTableLevelBarrier(region.getRegionInfo().getTable()); + } + } + + BulkLoadHFileResponse bulkLoad(RSRpcServices rsRpcServices, BulkLoadHFileRequest request) + throws ServiceException { + return rsRpcServices.bulkLoadHFileInternal(request); + } + + void increaseRowCacheSeqNum(HRegion region) { + region.increaseRowCacheSeqNum(); + } + + void removeTableLevelBarrier(TableName tableName) { + tableLevelBarrierMap.computeIfPresent(tableName, (k, counter) -> { + int remaining = counter.decrementAndGet(); + return (remaining <= 0) ? null : counter; + }); + } + + void createTableLevelBarrier(TableName tableName) { + tableLevelBarrierMap.computeIfAbsent(tableName, k -> new AtomicInteger(0)).incrementAndGet(); + } + + // @formatter:off + /** + * Row cache is only enabled when the following conditions are met: + * - Row cache is enabled at the table level. + * - Cache blocks is enabled in the get request. + * - A Get object cannot be distinguished from others except by its row key. + * So we check equality for the following: + * - filter + * - retrieving cells + * - TTL + * - attributes + * - CheckExistenceOnly + * - ColumnFamilyTimeRange + * - Consistency + * - MaxResultsPerColumnFamily + * - ReplicaId + * - RowOffsetPerColumnFamily + * @param get the Get request + * @param region the Region + * @return true if the row can be cached, false otherwise + */ + // @formatter:on + static boolean canCacheRow(Get get, Region region) { + return region.getTableDescriptor().isRowCacheEnabled() && get.getCacheBlocks() + && get.getFilter() == null && isRetrieveAllCells(get, region) && isDefaultTtl(region) + && get.getAttributesMap().isEmpty() && !get.isCheckExistenceOnly() + && get.getColumnFamilyTimeRange().isEmpty() && get.getConsistency() == Consistency.STRONG + && get.getMaxResultsPerColumnFamily() == -1 && get.getReplicaId() == -1 + && get.getRowOffsetPerColumnFamily() == 0 && get.getTimeRange().isAllTime(); + } + + private static boolean isRetrieveAllCells(Get get, Region region) { + if (region.getTableDescriptor().getColumnFamilyCount() != get.numFamilies()) { + return false; + } + + boolean hasQualifier = get.getFamilyMap().values().stream().anyMatch(Objects::nonNull); + return !hasQualifier; + } + + private static boolean isDefaultTtl(Region region) { + return Arrays.stream(region.getTableDescriptor().getColumnFamilies()) + .allMatch(cfd -> cfd.getTimeToLive() == ColumnFamilyDescriptorBuilder.DEFAULT_TTL); + } + + private R mutateWithRowCacheBarrier(HRegion region, List mutations, + RowOperation operation) throws IOException { + if (!region.getTableDescriptor().isRowCacheEnabled()) { + return operation.execute(); + } + + Set rowCacheKeys = new HashSet<>(mutations.size()); + try { + // Evict the entire row cache + mutations.forEach(mutation -> rowCacheKeys.add(new RowCacheKey(region, mutation.getRow()))); + rowCacheKeys.forEach(key -> { + // Creates a barrier that prevents the row cache from being populated for this row + // during mutation. Reads for the row can instead be served from HFiles or the block cache. + createRowLevelBarrier(key); + + // After creating the barrier, evict the existing row cache for this row, + // as it becomes invalid after the mutation + evictRowCache(region, key); + }); + + return execute(operation); + } finally { + // Remove the barrier after mutation to allow the row cache to be populated again + rowCacheKeys.forEach(this::removeRowLevelBarrier); + } + } + + private R mutateWithRowCacheBarrier(HRegion region, byte[] row, RowOperation operation) + throws IOException { + if (!region.getTableDescriptor().isRowCacheEnabled()) { + return operation.execute(); + } + + RowCacheKey key = new RowCacheKey(region, row); + try { + // Creates a barrier that prevents the row cache from being populated for this row + // during mutation. Reads for the row can instead be served from HFiles or the block cache. + createRowLevelBarrier(key); + + // After creating the barrier, evict the existing row cache for this row, + // as it becomes invalid after the mutation + evictRowCache(region, key); + + return execute(operation); + } finally { + // Remove the barrier after mutation to allow the row cache to be populated again + removeRowLevelBarrier(key); + } + } + + R execute(RowOperation operation) throws IOException { + return operation.execute(); + } + + void evictRowCache(HRegion region, RowCacheKey key) { + region.getBlockCache().evictBlock(key); + } + + /** + * Remove the barrier after mutation to allow the row cache to be populated again + * @param key the cache key of the row + */ + void removeRowLevelBarrier(RowCacheKey key) { + rowLevelBarrierMap.computeIfPresent(key, (k, counter) -> { + int remaining = counter.decrementAndGet(); + return (remaining <= 0) ? null : counter; + }); + } + + /** + * Creates a barrier to prevent the row cache from being populated for this row during mutation + * @param key the cache key of the row + */ + void createRowLevelBarrier(RowCacheKey key) { + rowLevelBarrierMap.computeIfAbsent(key, k -> new AtomicInteger(0)).incrementAndGet(); + } + + Result mutate(RSRpcServices rsRpcServices, HRegion region, ClientProtos.MutationProto mutation, + OperationQuota quota, CellScanner cellScanner, long nonceGroup, + ActivePolicyEnforcement spaceQuotaEnforcement, RpcCallContext context) throws IOException { + return mutateWithRowCacheBarrier(region, mutation.getRow().toByteArray(), + () -> rsRpcServices.mutateInternal(mutation, region, quota, cellScanner, nonceGroup, + spaceQuotaEnforcement, context)); + } + + CheckAndMutateResult checkAndMutate(HRegion region, CheckAndMutate checkAndMutate, + long nonceGroup, long nonce) throws IOException { + return mutateWithRowCacheBarrier(region, checkAndMutate.getRow(), + () -> region.checkAndMutate(checkAndMutate, nonceGroup, nonce)); + } + + CheckAndMutateResult checkAndMutate(HRegion region, List mutations, + CheckAndMutate checkAndMutate, long nonceGroup, long nonce) throws IOException { + return mutateWithRowCacheBarrier(region, mutations, + () -> region.checkAndMutate(checkAndMutate, nonceGroup, nonce)); + } + + OperationStatus[] batchMutate(HRegion region, Mutation[] mArray, boolean atomic, long nonceGroup, + long nonce) throws IOException { + return mutateWithRowCacheBarrier(region, Arrays.asList(mArray), + () -> region.batchMutate(mArray, atomic, nonceGroup, nonce)); + } + + // For testing only + AtomicInteger getRowLevelBarrier(RowCacheKey key) { + return rowLevelBarrierMap.get(key); + } + + // For testing only + AtomicInteger getTableLevelBarrier(TableName tableName) { + return tableLevelBarrierMap.get(tableName); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java new file mode 100644 index 000000000000..59e3271bcb2d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java @@ -0,0 +1,80 @@ +/* + * 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.regionserver; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.io.hfile.BlockType; +import org.apache.hadoop.hbase.io.hfile.Cacheable; +import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer; +import org.apache.hadoop.hbase.util.ClassSize; + +class RowCells implements Cacheable { + public static final long FIXED_OVERHEAD = ClassSize.estimateBase(RowCells.class, false); + + private final List cells = new ArrayList<>(); + + public RowCells(List cells) throws CloneNotSupportedException { + for (Cell cell : cells) { + if (!(cell instanceof ExtendedCell extCell)) { + throw new CloneNotSupportedException("Cell is not an ExtendedCell"); + } + try { + // To garbage collect the objects referenced by the cells + this.cells.add(extCell.deepClone()); + } catch (RuntimeException e) { + throw new CloneNotSupportedException("Deep clone failed"); + } + } + } + + @Override + public int getSerializedLength() { + // Cannot be serialized. Cached on heap only and cannot be moved off heap. + return 0; + } + + @Override + public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) { + // Cannot be serialized. Cached on heap only and cannot be moved off heap. + } + + @Override + public CacheableDeserializer getDeserializer() { + // Cannot be serialized. Cached on heap only and cannot be moved off heap. + return null; + } + + @Override + public BlockType getBlockType() { + return BlockType.ROW_CELLS; + } + + @Override + public long heapSize() { + long cellsSize = cells.stream().mapToLong(Cell::heapSize).sum(); + return FIXED_OVERHEAD + cellsSize; + } + + public List getCells() { + return cells; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java index b9bca1ba6b4e..49558b4b72a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java @@ -133,6 +133,12 @@ public void testCombinedBlockCacheStatsWithNoBlockType() throws Exception { testCombinedBlockCacheStats(null, 0, 1); } + @Test + public void testCombinedBlockCacheStatsWithRowCellsBlockType() throws Exception { + // ROW_CELLS type is cached only in the L1 cache, since it is not a DATA block type + testCombinedBlockCacheStats(BlockType.ROW_CELLS, 1, 0); + } + private CombinedBlockCache createCombinedBlockCache() { Configuration conf = UTIL.getConfiguration(); conf.set(BUCKET_CACHE_IOENGINE_KEY, "offheap"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowCacheKey.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowCacheKey.java new file mode 100644 index 000000000000..355f5ac612c3 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowCacheKey.java @@ -0,0 +1,114 @@ +/* + * 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.hfile; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotSame; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; + +@Category({ IOTests.class, SmallTests.class }) +public class TestRowCacheKey { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheKey.class); + + private static HRegion region1; + private static HRegion region2; + private static RegionInfo regionInfo1; + + @BeforeClass + public static void beforeClass() { + TableName tableName = TableName.valueOf("table1"); + + regionInfo1 = Mockito.mock(RegionInfo.class); + Mockito.when(regionInfo1.getEncodedName()).thenReturn("region1"); + Mockito.when(regionInfo1.getTable()).thenReturn(tableName); + + region1 = Mockito.mock(HRegion.class); + Mockito.when(region1.getRegionInfo()).thenReturn(regionInfo1); + + RegionInfo regionInfo2 = Mockito.mock(RegionInfo.class); + Mockito.when(regionInfo2.getEncodedName()).thenReturn("region2"); + Mockito.when(regionInfo2.getTable()).thenReturn(tableName); + + region2 = Mockito.mock(HRegion.class); + Mockito.when(region2.getRegionInfo()).thenReturn(regionInfo2); + } + + @Test + public void testEquality() { + RowCacheKey key11 = new RowCacheKey(region1, "row1".getBytes()); + RowCacheKey key12 = new RowCacheKey(region1, "row2".getBytes()); + RowCacheKey key21 = new RowCacheKey(region2, "row1".getBytes()); + RowCacheKey key22 = new RowCacheKey(region2, "row2".getBytes()); + RowCacheKey key11Another = new RowCacheKey(region1, "row1".getBytes()); + assertNotSame(key11, key11Another); + + // Ensure hashCode works well + assertNotEquals(key11.hashCode(), key12.hashCode()); + assertNotEquals(key11.hashCode(), key21.hashCode()); + assertNotEquals(key11.hashCode(), key22.hashCode()); + assertEquals(key11.hashCode(), key11Another.hashCode()); + + // Ensure equals works well + assertNotEquals(key11, key12); + assertNotEquals(key11, key21); + assertNotEquals(key11, key22); + assertEquals(key11, key11Another); + } + + @Test + public void testDifferentRowCacheSeqNum() { + RowCacheKey key1 = new RowCacheKey(region1, "row1".getBytes()); + + HRegion region1Another = Mockito.mock(HRegion.class); + Mockito.when(region1Another.getRegionInfo()).thenReturn(regionInfo1); + Mockito.when(region1Another.getRowCacheSeqNum()).thenReturn(1L); + RowCacheKey key1Another = new RowCacheKey(region1Another, "row1".getBytes()); + + assertNotEquals(key1.hashCode(), key1Another.hashCode()); + assertNotEquals(key1, key1Another); + } + + @Test + public void testHeapSize() { + RowCacheKey key; + long base = RowCacheKey.FIXED_OVERHEAD; + + key = new RowCacheKey(region1, "1".getBytes()); + assertEquals(base + 8, key.heapSize()); + + key = new RowCacheKey(region1, "12345678".getBytes()); + assertEquals(base + 8, key.heapSize()); + + key = new RowCacheKey(region1, "123456789".getBytes()); + assertEquals(base + 8 * 2, key.heapSize()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java index f1b6efe50a99..5dad6e21e5c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java @@ -612,6 +612,11 @@ public long getTrailerMissCount() { return 0; } + @Override + public long getRowMissCount() { + return 1; + } + @Override public long getDataHitCount() { return 0; @@ -662,6 +667,11 @@ public long getTrailerHitCount() { return 0; } + @Override + public long getRowHitCount() { + return 2; + } + @Override public int getSplitQueueSize() { return 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java index aac2a5922b9b..39ec4065769c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java @@ -124,8 +124,10 @@ public void testWrapperSource() { HELPER.assertGauge("blockCacheDataBlockCount", 300, serverSource); HELPER.assertGauge("blockCacheSize", 415, serverSource); HELPER.assertCounter("blockCacheHitCount", 416, serverSource); + HELPER.assertCounter("blockCacheRowHitCount", 2, serverSource); HELPER.assertCounter("blockCacheHitCachingCount", 16, serverSource); HELPER.assertCounter("blockCacheMissCount", 417, serverSource); + HELPER.assertCounter("blockCacheRowMissCount", 1, serverSource); HELPER.assertCounter("blockCacheMissCachingCount", 17, serverSource); HELPER.assertCounter("blockCacheEvictionCount", 418, serverSource); HELPER.assertGauge("blockCacheCountHitPercent", 98, serverSource); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java new file mode 100644 index 000000000000..ce6545ea6c3e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java @@ -0,0 +1,481 @@ +/* + * 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.regionserver; + +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.CheckAndMutate; +import org.apache.hadoop.hbase.client.CheckAndMutateResult; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Row; +import org.apache.hadoop.hbase.client.RowMutations; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.io.hfile.RowCacheKey; +import org.apache.hadoop.hbase.test.MetricsAssertHelper; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestRowCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCache.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final byte[] CF1 = Bytes.toBytes("cf1"); + private static final byte[] CF2 = Bytes.toBytes("cf2"); + private static final byte[] Q1 = Bytes.toBytes("q1"); + private static final byte[] Q2 = Bytes.toBytes("q2"); + + private static MetricsAssertHelper metricsHelper; + private static MetricsRegionServer metricsRegionServer; + private static MetricsRegionServerSource serverSource; + + private static Admin admin; + + private TableName tableName; + private Table table; + private final Map counterBase = new HashMap<>(); + + @Rule + public TestName testName = new TestName(); + + @BeforeClass + public static void beforeClass() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + // To test simply, regardless of the number of HFiles + conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); + SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); + cluster.waitForActiveAndReadyMaster(); + admin = TEST_UTIL.getAdmin(); + + metricsHelper = CompatibilityFactory.getInstance(MetricsAssertHelper.class); + metricsRegionServer = cluster.getRegionServer(0).getMetrics(); + serverSource = metricsRegionServer.getMetricsSource(); + } + + @AfterClass + public static void afterClass() throws Exception { + HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false; + TEST_UTIL.shutdownMiniCluster(); + } + + @Before + public void beforeTestMethod() throws Exception { + ColumnFamilyDescriptor cf1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + // To test data block encoding + ColumnFamilyDescriptor cf2 = ColumnFamilyDescriptorBuilder.newBuilder(CF2) + .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF).build(); + + tableName = TableName.valueOf(testName.getMethodName()); + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName).setRowCacheEnabled(true) + .setColumnFamily(cf1).setColumnFamily(cf2).build(); + admin.createTable(td); + table = admin.getConnection().getTable(tableName); + } + + @After + public void afterTestMethod() throws Exception { + counterBase.clear(); + + admin.disableTable(tableName); + admin.deleteTable(tableName); + } + + private void setCounterBase(String metric, long value) { + counterBase.put(metric, value); + } + + private void assertCounterDiff(String metric, long diff) { + Long base = counterBase.get(metric); + if (base == null) { + throw new IllegalStateException( + "base counter of " + metric + " metric should have been set before by setCounterBase()"); + } + long newValue = base + diff; + metricsHelper.assertCounter(metric, newValue, serverSource); + counterBase.put(metric, newValue); + } + + private static void recomputeMetrics() { + metricsRegionServer.getRegionServerWrapper().forceRecompute(); + } + + @Test + public void testGetWithRowCache() throws IOException, InterruptedException { + byte[] rowKey = "row".getBytes(); + Get get = new Get(rowKey); + Result result; + + HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + RowCacheKey rowCacheKey = new RowCacheKey(region, rowKey); + + // Put a row + Put put = new Put(rowKey); + put.addColumn(CF1, Q1, Bytes.toBytes(0L)); + put.addColumn(CF1, Q2, "12".getBytes()); + put.addColumn(CF2, Q1, "21".getBytes()); + put.addColumn(CF2, Q2, "22".getBytes()); + table.put(put); + + // Initialize metrics + recomputeMetrics(); + setCounterBase("Get_num_ops", metricsHelper.getCounter("Get_num_ops", serverSource)); + setCounterBase("blockCacheRowHitCount", + metricsHelper.getCounter("blockCacheRowHitCount", serverSource)); + + // First get to populate the row cache + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertArrayEquals(Bytes.toBytes(0L), result.getValue(CF1, Q1)); + assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); + assertArrayEquals("21".getBytes(), result.getValue(CF2, Q1)); + assertArrayEquals("22".getBytes(), result.getValue(CF2, Q2)); + assertCounterDiff("Get_num_ops", 1); + // Ensure the get operation from HFile without row cache + assertCounterDiff("blockCacheRowHitCount", 0); + + // Get from the row cache + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertArrayEquals(Bytes.toBytes(0L), result.getValue(CF1, Q1)); + assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); + assertArrayEquals("21".getBytes(), result.getValue(CF2, Q1)); + assertArrayEquals("22".getBytes(), result.getValue(CF2, Q2)); + assertCounterDiff("Get_num_ops", 1); + // Ensure the get operation from the row cache + assertCounterDiff("blockCacheRowHitCount", 1); + + // Row cache is invalidated by the put operation + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + table.put(put); + assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + + // Get is executed without the row cache; however, the cache is re-populated as a result + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertCounterDiff("Get_num_ops", 1); + // Ensure the get operation not from the row cache + assertCounterDiff("blockCacheRowHitCount", 0); + + // Get again with the row cache + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertCounterDiff("Get_num_ops", 1); + // Ensure the get operation from the row cache + assertCounterDiff("blockCacheRowHitCount", 1); + + // Row cache is invalidated by the increment operation + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + table.incrementColumnValue(rowKey, CF1, Q1, 1); + assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + + // Get is executed without the row cache; however, the cache is re-populated as a result + table.get(get); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + + // Row cache is invalidated by the append operation + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + Append append = new Append(rowKey); + append.addColumn(CF1, Q1, Bytes.toBytes(0L)); + table.append(append); + assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + + // Get is executed without the row cache; however, the cache is re-populated as a result + table.get(get); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + + // Row cache is invalidated by the delete operation + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + Delete delete = new Delete(rowKey); + delete.addColumn(CF1, Q1); + table.delete(delete); + assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + } + + @Test(expected = DoNotRetryIOException.class) + public void testPutWithTTL() throws IOException { + // Put with TTL is not allowed on tables with row cache enabled, because cached rows cannot + // track TTL expiration + Put put = new Put("row".getBytes()); + put.addColumn(CF1, Q1, "11".getBytes()); + put.setTTL(1); + table.put(put); + } + + @Test + public void testCheckAndMutate() throws IOException, InterruptedException { + byte[] rowKey = "row".getBytes(); + Get get = new Get(rowKey); + Result result; + CheckAndMutate cam; + CheckAndMutateResult camResult; + + HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + RowCacheKey rowCacheKey = new RowCacheKey(region, rowKey); + + // Put a row + Put put1 = new Put(rowKey); + put1.addColumn(CF1, Q1, "11".getBytes()); + put1.addColumn(CF1, Q2, "12".getBytes()); + table.put(put1); + + // Validate that the row cache is populated + result = table.get(get); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); + assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); + + // The row cache is not invalidated when a checkAndMutate operation fails + Put put2 = new Put(rowKey); + put2.addColumn(CF1, Q2, "1212".getBytes()); + cam = CheckAndMutate.newBuilder(rowKey).ifEquals(CF1, Q2, "00".getBytes()).build(put2); + camResult = table.checkAndMutate(cam); + assertFalse(camResult.isSuccess()); + assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + + // Validate that the row cache is populated + result = table.get(get); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); + assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); + + // The row cache is invalidated by a checkAndMutate operation + cam = CheckAndMutate.newBuilder(rowKey).ifEquals(CF1, Q2, "12".getBytes()).build(put2); + camResult = table.checkAndMutate(cam); + assertTrue(camResult.isSuccess()); + assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + } + + @Test + public void testCheckAndMutates() throws IOException, InterruptedException { + byte[] rowKey1 = "row1".getBytes(); + byte[] rowKey2 = "row2".getBytes(); + Get get1 = new Get(rowKey1); + Get get2 = new Get(rowKey2); + Result result1, result2; + List cams; + List camResults; + + HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + RowCacheKey rowCacheKey1 = new RowCacheKey(region, rowKey1); + RowCacheKey rowCacheKey2 = new RowCacheKey(region, rowKey2); + + // Put rows + Put put1 = new Put(rowKey1); + put1.addColumn(CF1, Q1, "111".getBytes()); + put1.addColumn(CF1, Q2, "112".getBytes()); + table.put(put1); + Put put2 = new Put(rowKey2); + put2.addColumn(CF1, Q1, "211".getBytes()); + put2.addColumn(CF1, Q2, "212".getBytes()); + table.put(put2); + + // Validate that the row caches are populated + result1 = table.get(get1); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertArrayEquals("111".getBytes(), result1.getValue(CF1, Q1)); + assertArrayEquals("112".getBytes(), result1.getValue(CF1, Q2)); + result2 = table.get(get2); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); + assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); + + // The row caches are invalidated by checkAndMutate operations + cams = new ArrayList<>(); + cams.add(CheckAndMutate.newBuilder(rowKey1).ifEquals(CF1, Q2, "112".getBytes()).build(put1)); + cams.add(CheckAndMutate.newBuilder(rowKey2).ifEquals(CF1, Q2, "212".getBytes()).build(put2)); + camResults = table.checkAndMutate(cams); + assertTrue(camResults.get(0).isSuccess()); + assertTrue(camResults.get(1).isSuccess()); + assertNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + } + + @Test + public void testRowMutations() throws IOException, InterruptedException { + byte[] rowKey1 = "row1".getBytes(); + byte[] rowKey2 = "row2".getBytes(); + Get get1 = new Get(rowKey1); + Get get2 = new Get(rowKey2); + Result result1, result2; + + HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + RowCacheKey rowCacheKey1 = new RowCacheKey(region, rowKey1); + RowCacheKey rowCacheKey2 = new RowCacheKey(region, rowKey2); + + // Put rows + Put put1 = new Put(rowKey1); + put1.addColumn(CF1, Q1, "111".getBytes()); + put1.addColumn(CF1, Q2, "112".getBytes()); + table.put(put1); + Put put2 = new Put(rowKey2); + put2.addColumn(CF1, Q1, "211".getBytes()); + put2.addColumn(CF1, Q2, "212".getBytes()); + table.put(put2); + + // Validate that the row caches are populated + result1 = table.get(get1); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertArrayEquals("111".getBytes(), result1.getValue(CF1, Q1)); + assertArrayEquals("112".getBytes(), result1.getValue(CF1, Q2)); + result2 = table.get(get2); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); + assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); + + // The row caches are invalidated by batch operation + Put put12 = new Put(rowKey1); + put12.addColumn(CF1, Q1, "111111".getBytes()); + Put put13 = new Put(rowKey1); + put13.addColumn(CF1, Q2, "112112".getBytes()); + RowMutations rms = new RowMutations(rowKey1); + rms.add(put12); + rms.add(put13); + CheckAndMutate cam = + CheckAndMutate.newBuilder(rowKey1).ifEquals(CF1, Q1, "111".getBytes()).build(rms); + table.checkAndMutate(cam); + assertNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + + // Validate that the row caches are populated + result1 = table.get(get1); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertArrayEquals("111111".getBytes(), result1.getValue(CF1, Q1)); + assertArrayEquals("112112".getBytes(), result1.getValue(CF1, Q2)); + result2 = table.get(get2); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); + assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); + } + + @Test + public void testBatch() throws IOException, InterruptedException { + byte[] rowKey1 = "row1".getBytes(); + byte[] rowKey2 = "row2".getBytes(); + byte[] rowKey3 = "row3".getBytes(); + Get get1 = new Get(rowKey1); + Get get2 = new Get(rowKey2); + Get get3 = new Get(rowKey3); + List batchOperations; + Object[] results; + + HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + RowCacheKey rowCacheKey1 = new RowCacheKey(region, rowKey1); + RowCacheKey rowCacheKey2 = new RowCacheKey(region, rowKey2); + RowCacheKey rowCacheKey3 = new RowCacheKey(region, rowKey3); + + // Put rows + batchOperations = new ArrayList<>(); + Put put1 = new Put(rowKey1); + put1.addColumn(CF1, Q1, "111".getBytes()); + put1.addColumn(CF1, Q2, "112".getBytes()); + batchOperations.add(put1); + Put put2 = new Put(rowKey2); + put2.addColumn(CF1, Q1, "211".getBytes()); + put2.addColumn(CF1, Q2, "212".getBytes()); + batchOperations.add(put2); + Put put3 = new Put(rowKey3); + put3.addColumn(CF1, Q1, "311".getBytes()); + put3.addColumn(CF1, Q2, "312".getBytes()); + batchOperations.add(put3); + results = new Result[batchOperations.size()]; + table.batch(batchOperations, results); + + // Validate that the row caches are populated + batchOperations = new ArrayList<>(); + batchOperations.add(get1); + batchOperations.add(get2); + batchOperations.add(get3); + results = new Object[batchOperations.size()]; + table.batch(batchOperations, results); + assertEquals(3, results.length); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertArrayEquals("111".getBytes(), ((Result) results[0]).getValue(CF1, Q1)); + assertArrayEquals("112".getBytes(), ((Result) results[0]).getValue(CF1, Q2)); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + assertArrayEquals("211".getBytes(), ((Result) results[1]).getValue(CF1, Q1)); + assertArrayEquals("212".getBytes(), ((Result) results[1]).getValue(CF1, Q2)); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey3, false, false, false)); + assertArrayEquals("311".getBytes(), ((Result) results[2]).getValue(CF1, Q1)); + assertArrayEquals("312".getBytes(), ((Result) results[2]).getValue(CF1, Q2)); + + // The row caches are invalidated by batch operation + batchOperations = new ArrayList<>(); + batchOperations.add(put1); + Put put2New = new Put(rowKey2); + put2New.addColumn(CF1, Q1, "211211".getBytes()); + put2New.addColumn(CF1, Q2, "212".getBytes()); + CheckAndMutate cam = + CheckAndMutate.newBuilder(rowKey2).ifEquals(CF1, Q1, "211".getBytes()).build(put2New); + batchOperations.add(cam); + results = new Object[batchOperations.size()]; + table.batch(batchOperations, results); + assertEquals(2, results.length); + assertNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey3, false, false, false)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java new file mode 100644 index 000000000000..83ce32ee5511 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java @@ -0,0 +1,216 @@ +/* + * 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.regionserver; + +import java.io.IOException; +import java.util.function.Function; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Consistency; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.IsolationLevel; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.RowFilter; +import org.apache.hadoop.hbase.security.visibility.Authorizations; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestRowCacheCanCacheRow { + private static final byte[] CF1 = "cf1".getBytes(); + private static final byte[] CF2 = "cf2".getBytes(); + private static final byte[] ROW_KEY = "row".getBytes(); + private static final TableName TABLE_NAME = TableName.valueOf("test"); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheCanCacheRow.class); + + @Test + public void testRowCacheEnabled() { + Region region = Mockito.mock(Region.class); + ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + TableDescriptor td; + + Get get = new Get(ROW_KEY); + get.addFamily(CF1); + + td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true).setColumnFamily(cfd) + .build(); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + + // Disable row cache, expect false + td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(cfd) + .setRowCacheEnabled(false).build(); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + } + + @Test + public void testRetrieveAllCells() { + Region region = Mockito.mock(Region.class); + ColumnFamilyDescriptor cfd1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + ColumnFamilyDescriptor cfd2 = ColumnFamilyDescriptorBuilder.newBuilder(CF2).build(); + TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) + .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + + // Not all CFs, expect false + Get get = new Get(ROW_KEY); + get.addFamily(CF1); + Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + + // All CFs, expect true + get.addFamily(CF2); + Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + + // Not all qualifiers, expect false + get.addColumn(CF1, "q1".getBytes()); + Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + } + + @Test + public void testTtl() { + ColumnFamilyDescriptor cfd1; + ColumnFamilyDescriptor cfd2; + TableDescriptor td; + Region region = Mockito.mock(Region.class); + Get get = new Get(ROW_KEY); + get.addFamily(CF1); + get.addFamily(CF2); + + // Ttl is set, expect false + cfd1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).setTimeToLive(1).build(); + cfd2 = ColumnFamilyDescriptorBuilder.newBuilder(CF2).build(); + td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) + .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + + // Ttl is not set, expect true + cfd1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) + .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + } + + @Test + public void testFilter() { + testWith( + get -> get.setFilter(new RowFilter(CompareOperator.EQUAL, new BinaryComparator(ROW_KEY)))); + } + + @Test + public void testCacheBlock() { + testWith(get -> get.setCacheBlocks(false)); + } + + @Test + public void testAttribute() { + testWith(get -> get.setAttribute("test", "value".getBytes())); + } + + @Test + public void testCheckExistenceOnly() { + testWith(get -> get.setCheckExistenceOnly(true)); + } + + @Test + public void testColumnFamilyTimeRange() { + testWith(get -> get.setColumnFamilyTimeRange(CF1, 1000, 2000)); + } + + @Test + public void testConsistency() { + testWith(get -> get.setConsistency(Consistency.TIMELINE)); + } + + @Test + public void testAuthorizations() { + testWith(get -> get.setAuthorizations(new Authorizations("foo"))); + } + + @Test + public void testId() { + testWith(get -> get.setId("test")); + } + + @Test + public void testIsolationLevel() { + testWith(get -> get.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED)); + } + + @Test + public void testMaxResultsPerColumnFamily() { + testWith(get -> get.setMaxResultsPerColumnFamily(2)); + } + + @Test + public void testReplicaId() { + testWith(get -> get.setReplicaId(1)); + } + + @Test + public void testRowOffsetPerColumnFamily() { + testWith(get -> get.setRowOffsetPerColumnFamily(1)); + } + + @Test + public void testTimeRange() { + testWith(get -> { + try { + return get.setTimeRange(1, 2); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testTimestamp() { + testWith(get -> get.setTimestamp(1)); + } + + private static void testWith(Function func) { + Region region = Mockito.mock(Region.class); + ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) + .setColumnFamily(cfd).build(); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + + Get get = new Get(ROW_KEY); + get.addFamily(CF1); + Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + + // expect false + func.apply(get); + Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java new file mode 100644 index 000000000000..4b528d3b5e62 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java @@ -0,0 +1,122 @@ +/* + * 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.regionserver; + +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.io.hfile.RowCacheKey; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@Category(MediumTests.class) +@RunWith(Parameterized.class) +public class TestRowCacheClearRegionBlockCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheClearRegionBlockCache.class); + + private static final byte[] FAMILY = Bytes.toBytes("family"); + private static final byte[][] SPLIT_KEY = new byte[][] { Bytes.toBytes("5") }; + private static final int NUM_RS = 2; + + private final HBaseTestingUtil HTU = new HBaseTestingUtil(); + + private Admin admin; + private Table table; + private TableName tableName; + + @Rule + public TestName testName = new TestName(); + + @Parameterized.Parameter + public String cacheType; + + @Parameterized.Parameters(name = "{index}: {0}") + public static Object[] data() { + return new Object[] { "lru", "bucket" }; + } + + @Before + public void setup() throws Exception { + if (cacheType.equals("bucket")) { + Configuration conf = HTU.getConfiguration(); + conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap"); + conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 30); + } + + // To test simply, regardless of the number of HFiles + HTU.getConfiguration().setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); + HTU.startMiniCluster(NUM_RS); + admin = HTU.getAdmin(); + + // Create table + tableName = TableName.valueOf(testName.getMethodName().replaceAll("[\\[\\]: ]", "_")); + ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build(); + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(cfd) + .setRowCacheEnabled(true).build(); + admin.createTable(td, SPLIT_KEY); + table = admin.getConnection().getTable(tableName); + + HTU.loadNumericRows(table, FAMILY, 1, 10); + } + + @After + public void teardown() throws Exception { + HTU.shutdownMiniCluster(); + } + + @Test + public void testClearRowCache() throws Exception { + HRegion region = HTU.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + + Get get = new Get(Bytes.toBytes("1")); + table.get(get); + + // Ensure a row cache entry exists + RowCacheKey rowCacheKey = new RowCacheKey(region, Bytes.toBytes("1")); + assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + + // Ensure the row cache is cleared + admin.clearBlockCache(tableName); + assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java new file mode 100644 index 000000000000..2d29bc8796d4 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java @@ -0,0 +1,204 @@ +/* + * 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.regionserver; + +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; +import static org.junit.Assert.assertArrayEquals; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CompatibilityFactory; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.test.MetricsAssertHelper; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestRowCacheConfigurationObserver { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheConfigurationObserver.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final byte[] CF1 = Bytes.toBytes("cf1"); + private static final byte[] Q1 = Bytes.toBytes("q1"); + + private static MetricsAssertHelper metricsHelper; + private static MetricsRegionServer metricsRegionServer; + private static MetricsRegionServerSource serverSource; + + private static Admin admin; + + private TableName tableName; + private Table table; + private final Map counterBase = new HashMap<>(); + + @Rule + public TestName testName = new TestName(); + + @BeforeClass + public static void beforeClass() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + // Set a value different from the default + conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); + SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); + cluster.waitForActiveAndReadyMaster(); + admin = TEST_UTIL.getAdmin(); + + metricsHelper = CompatibilityFactory.getInstance(MetricsAssertHelper.class); + metricsRegionServer = cluster.getRegionServer(0).getMetrics(); + serverSource = metricsRegionServer.getMetricsSource(); + } + + @AfterClass + public static void afterClass() throws Exception { + HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false; + TEST_UTIL.shutdownMiniCluster(); + } + + @Before + public void beforeTestMethod() throws Exception { + ColumnFamilyDescriptor cf1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + + tableName = TableName.valueOf(testName.getMethodName()); + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName).setRowCacheEnabled(true) + .setColumnFamily(cf1).build(); + admin.createTable(td); + table = admin.getConnection().getTable(tableName); + } + + @After + public void afterTestMethod() throws Exception { + counterBase.clear(); + + admin.disableTable(tableName); + admin.deleteTable(tableName); + } + + private void setCounterBase(String metric, long value) { + counterBase.put(metric, value); + } + + private void assertCounterDiff(String metric, long diff) { + Long base = counterBase.get(metric); + if (base == null) { + throw new IllegalStateException( + "base counter of " + metric + " metric should have been set before by setCounterBase()"); + } + long newValue = base + diff; + metricsHelper.assertCounter(metric, newValue, serverSource); + counterBase.put(metric, newValue); + } + + private static void recomputeMetrics() { + metricsRegionServer.getRegionServerWrapper().forceRecompute(); + } + + /** + * If the number of HFiles is below the configured minimum, row caching has no effect. + */ + @Test + public void testRowCacheWithHFilesCount() throws IOException { + // Change ROW_CACHE_ACTIVATE_MIN_HFILES_KEY online + Configuration conf = TEST_UTIL.getConfiguration(); + conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 2); + for (ServerName serverName : admin.getRegionServers()) { + HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(serverName); + regionServer.getConfigurationManager().notifyAllObservers(conf); + } + + byte[] rowKey = "row".getBytes(); + Get get = new Get(rowKey); + Result result; + + // Initialize metrics + recomputeMetrics(); + setCounterBase("Get_num_ops", metricsHelper.getCounter("Get_num_ops", serverSource)); + setCounterBase("blockCacheRowHitCount", + metricsHelper.getCounter("blockCacheRowHitCount", serverSource)); + + Put put = new Put(rowKey); + put.addColumn(CF1, Q1, "11".getBytes()); + table.put(put); + + // The row cache is not populated yet, as the store file count is 0 + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); + assertCounterDiff("Get_num_ops", 1); + assertCounterDiff("blockCacheRowHitCount", 0); + + // Flush, 1 store file exists + TEST_UTIL.getAdmin().flush(tableName); + + // The row cache is not populated yet, as the store file count is 1 + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); + assertCounterDiff("Get_num_ops", 1); + assertCounterDiff("blockCacheRowHitCount", 0); + + // Flush, 2(ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT) store files exist + table.put(put); + TEST_UTIL.getAdmin().flush(tableName); + + // The row cache is populated now, as the store file count is 2. + // But the row cache is not hit yet, it will be hit only after this operation. + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); + assertCounterDiff("Get_num_ops", 1); + assertCounterDiff("blockCacheRowHitCount", 0); + + // Get again, this time the row cache is hit + result = table.get(get); + recomputeMetrics(); + assertArrayEquals(rowKey, result.getRow()); + assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); + assertCounterDiff("Get_num_ops", 1); + assertCounterDiff("blockCacheRowHitCount", 1); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheHRegion.java new file mode 100644 index 000000000000..a8c59dc6ccbc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheHRegion.java @@ -0,0 +1,97 @@ +/* + * 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.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestRowCacheHRegion { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheHRegion.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + public static final byte[] CF = Bytes.toBytes("cf1"); + + @Rule + public TestName currentTest = new TestName(); + + @BeforeClass + public static void setupCluster() throws Exception { + TEST_UTIL.startMiniCluster(1); + } + + @AfterClass + public static void teardownCluster() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testOpenHRegion() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + WALFactory walFactory = new WALFactory(conf, + ServerName.valueOf(currentTest.getMethodName(), 16010, EnvironmentEdgeManager.currentTime()) + .toString()); + WAL wal = walFactory.getWAL(null); + Path hbaseRootDir = CommonFSUtils.getRootDir(conf); + TableName tableName = TableName.valueOf(currentTest.getMethodName()); + RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build(); + TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build(); + HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0); + HRegion region = HRegion.openHRegion(conf, FileSystem.get(conf), hbaseRootDir, hri, htd, wal, + regionServer, null); + + // Verify that rowCacheSeqNum is initialized correctly + assertNotEquals(HConstants.NO_SEQNUM, region.getRowCacheSeqNum()); + assertEquals(region.getOpenSeqNum(), region.getRowCacheSeqNum()); + + region.close(); + walFactory.close(); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java new file mode 100644 index 000000000000..49908704fba4 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java @@ -0,0 +1,263 @@ +/* + * 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.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.CheckAndMutate; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.io.hfile.BlockCache; +import org.apache.hadoop.hbase.io.hfile.LruBlockCache; +import org.apache.hadoop.hbase.io.hfile.RowCacheKey; +import org.apache.hadoop.hbase.ipc.RpcCallContext; +import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement; +import org.apache.hadoop.hbase.quotas.OperationQuota; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.InOrder; +import org.mockito.Mockito; + +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestRowCacheService { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheService.class); + + @Test + public void testBarrier() throws IOException { + // Mocking dependencies to create RowCacheService instance + RegionInfo regionInfo = Mockito.mock(RegionInfo.class); + Mockito.when(regionInfo.getEncodedName()).thenReturn("region1"); + TableName tableName = TableName.valueOf("table1"); + Mockito.when(regionInfo.getTable()).thenReturn(tableName); + + List stores = new ArrayList<>(); + HStore hStore = Mockito.mock(HStore.class); + Mockito.when(hStore.getStorefilesCount()).thenReturn(2); + stores.add(hStore); + + BlockCache blockCache = new LruBlockCache(1024, 64 * 1024); + + ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder("CF1".getBytes()).build(); + TableDescriptor td = Mockito.mock(TableDescriptor.class); + Mockito.when(td.isRowCacheEnabled()).thenReturn(true); + Mockito.when(td.getColumnFamilies()).thenReturn(new ColumnFamilyDescriptor[] { cfd }); + + byte[] rowKey = "row".getBytes(); + Get get = new Get(rowKey); + Scan scan = new Scan(get); + + RegionScannerImpl regionScanner = Mockito.mock(RegionScannerImpl.class); + + HRegion region = Mockito.mock(HRegion.class); + Mockito.when(region.getRegionInfo()).thenReturn(regionInfo); + Mockito.when(region.getBlockCache()).thenReturn(blockCache); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + Mockito.when(region.getStores()).thenReturn(stores); + Mockito.when(region.getScanner(scan)).thenReturn(regionScanner); + + RowCacheKey key = new RowCacheKey(region, rowKey); + List results = new ArrayList<>(); + results.add(KeyValueTestUtil.create("row", "CF", "q1", 1, "v1")); + + // Initialize RowCacheService + Configuration conf = HBaseConfiguration.create(); + RowCacheService rowCacheService = new RowCacheService(conf); + + // Verify that row cache populated before creating a row level barrier + rowCacheService.getScanner(region, get, scan, results); + assertNotNull(blockCache.getBlock(key, true, false, true)); + assertNull(rowCacheService.getRowLevelBarrier(key)); + + // Evict the row cache + region.getBlockCache().evictBlock(key); + assertNull(blockCache.getBlock(key, true, false, true)); + + // Create a row level barrier for the row key + rowCacheService.createRowLevelBarrier(key); + assertEquals(1, rowCacheService.getRowLevelBarrier(key).get()); + + // Verify that no row cache populated after creating a row level barrier + rowCacheService.getScanner(region, get, scan, results); + assertNull(blockCache.getBlock(key, true, false, true)); + + // Remove the row level barrier + rowCacheService.removeRowLevelBarrier(key); + assertNull(rowCacheService.getRowLevelBarrier(key)); + + // Verify that row cache populated before creating a table level barrier + rowCacheService.getScanner(region, get, scan, results); + assertNotNull(blockCache.getBlock(key, true, false, true)); + assertNull(rowCacheService.getTableLevelBarrier(tableName)); + + // Evict the row cache + region.getBlockCache().evictBlock(key); + assertNull(blockCache.getBlock(key, true, false, true)); + + // Create a table level barrier for the row key + rowCacheService.createTableLevelBarrier(tableName); + assertEquals(1, rowCacheService.getTableLevelBarrier(tableName).get()); + + // Verify that no row cache populated after creating a table level barrier + rowCacheService.getScanner(region, get, scan, results); + assertNull(blockCache.getBlock(key, true, false, true)); + + // Remove the table level barrier + rowCacheService.removeTableLevelBarrier(tableName); + assertNull(rowCacheService.getTableLevelBarrier(tableName)); + } + + @Test + public void testMutate() throws IOException, ServiceException { + // Mocking RowCacheService and its dependencies + TableDescriptor tableDescriptor = Mockito.mock(TableDescriptor.class); + Mockito.when(tableDescriptor.isRowCacheEnabled()).thenReturn(true); + + RegionInfo regionInfo = Mockito.mock(RegionInfo.class); + Mockito.when(regionInfo.getEncodedName()).thenReturn("region1"); + + HRegion region = Mockito.mock(HRegion.class); + Mockito.when(region.getTableDescriptor()).thenReturn(tableDescriptor); + Mockito.when(region.getRegionInfo()).thenReturn(regionInfo); + Mockito.when(region.getBlockCache()).thenReturn(Mockito.mock(BlockCache.class)); + + RSRpcServices rsRpcServices = Mockito.mock(RSRpcServices.class); + Mockito.when(rsRpcServices.getRegion(Mockito.any())).thenReturn(region); + + MutationProto mutationProto = MutationProto.newBuilder().build(); + + OperationQuota operationQuota = Mockito.mock(OperationQuota.class); + + CellScanner cellScanner = Mockito.mock(CellScanner.class); + + ActivePolicyEnforcement activePolicyEnforcement = Mockito.mock(ActivePolicyEnforcement.class); + + RpcCallContext rpcCallContext = Mockito.mock(RpcCallContext.class); + + CheckAndMutate checkAndMutate = CheckAndMutate.newBuilder("row".getBytes()) + .ifEquals("CF".getBytes(), "q1".getBytes(), "v1".getBytes()).build(new Put("row".getBytes())); + + Put put1 = new Put("row1".getBytes()); + put1.addColumn("CF".getBytes(), "q1".getBytes(), "v1".getBytes()); + Put put2 = new Put("row1".getBytes()); + put2.addColumn("CF".getBytes(), "q1".getBytes(), "v1".getBytes()); + List mutations = new ArrayList<>(); + mutations.add(put1); + mutations.add(put2); + + Mutation[] mutationArray = new Mutation[mutations.size()]; + mutations.toArray(mutationArray); + + RowCacheService rowCacheService; + InOrder inOrder; + + // Mutate + rowCacheService = Mockito.mock(RowCacheService.class); + Mockito.when(rowCacheService.mutate(rsRpcServices, region, mutationProto, operationQuota, + cellScanner, 0, activePolicyEnforcement, rpcCallContext)).thenCallRealMethod(); + inOrder = Mockito.inOrder(rowCacheService); + rowCacheService.mutate(rsRpcServices, region, mutationProto, operationQuota, cellScanner, 0, + activePolicyEnforcement, rpcCallContext); + // Verify the sequence of method calls + inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); + + // CheckAndMutate + rowCacheService = Mockito.mock(RowCacheService.class); + Mockito.when(rowCacheService.checkAndMutate(region, checkAndMutate, 0, 0)).thenCallRealMethod(); + inOrder = Mockito.inOrder(rowCacheService); + rowCacheService.checkAndMutate(region, checkAndMutate, 0, 0); + // Verify the sequence of method calls + inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); + + // RowMutations + rowCacheService = Mockito.mock(RowCacheService.class); + Mockito.when(rowCacheService.checkAndMutate(region, mutations, checkAndMutate, 0, 0)) + .thenCallRealMethod(); + inOrder = Mockito.inOrder(rowCacheService); + rowCacheService.checkAndMutate(region, mutations, checkAndMutate, 0, 0); + // Verify the sequence of method calls + inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); + + // Batch + rowCacheService = Mockito.mock(RowCacheService.class); + Mockito.when(rowCacheService.batchMutate(region, mutationArray, true, 0, 0)) + .thenCallRealMethod(); + inOrder = Mockito.inOrder(rowCacheService); + rowCacheService.batchMutate(region, mutationArray, true, 0, 0); + // Verify the sequence of method calls + inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); + + // Bulkload + HBaseProtos.RegionSpecifier regionSpecifier = HBaseProtos.RegionSpecifier.newBuilder() + .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME) + .setValue(ByteString.copyFrom("region".getBytes())).build(); + ClientProtos.BulkLoadHFileRequest bulkLoadRequest = + ClientProtos.BulkLoadHFileRequest.newBuilder().setRegion(regionSpecifier).build(); + rowCacheService = Mockito.mock(RowCacheService.class); + Mockito.when(rowCacheService.bulkLoadHFile(rsRpcServices, bulkLoadRequest)) + .thenCallRealMethod(); + inOrder = Mockito.inOrder(rowCacheService); + rowCacheService.bulkLoadHFile(rsRpcServices, bulkLoadRequest); + // Verify the sequence of method calls + inOrder.verify(rowCacheService, Mockito.times(1)).createTableLevelBarrier(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).increaseRowCacheSeqNum(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).bulkLoad(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).removeTableLevelBarrier(Mockito.any()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCells.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCells.java new file mode 100644 index 000000000000..6307bbce3059 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCells.java @@ -0,0 +1,74 @@ +/* + * 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.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; + +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestRowCells { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCells.class); + + @Test + public void testDeepClone() throws CloneNotSupportedException { + List cells = new ArrayList<>(); + KeyValue kv1 = KeyValueTestUtil.create("row", "CF", "q1", 1, "v1"); + cells.add(kv1); + KeyValue kv2 = KeyValueTestUtil.create("row", "CF", "q12", 2, "v22"); + cells.add(kv2); + RowCells rowCells = new RowCells(cells); + + // Ensure deep clone happened + assertNotSame(kv1, rowCells.getCells().get(0)); + assertEquals(kv1, rowCells.getCells().get(0)); + assertNotSame(kv2, rowCells.getCells().get(1)); + assertEquals(kv2, rowCells.getCells().get(1)); + } + + @Test + public void testHeapSize() throws CloneNotSupportedException { + List cells; + RowCells rowCells; + + cells = new ArrayList<>(); + rowCells = new RowCells(cells); + assertEquals(RowCells.FIXED_OVERHEAD, rowCells.heapSize()); + + cells = new ArrayList<>(); + KeyValue kv1 = KeyValueTestUtil.create("row", "CF", "q1", 1, "v1"); + cells.add(kv1); + KeyValue kv2 = KeyValueTestUtil.create("row", "CF", "q22", 2, "v22"); + cells.add(kv2); + rowCells = new RowCells(cells); + assertEquals(RowCells.FIXED_OVERHEAD + kv1.heapSize() + kv2.heapSize(), rowCells.heapSize()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java new file mode 100644 index 000000000000..61ed58f28110 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java @@ -0,0 +1,160 @@ +/* + * 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.tool; + +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.io.IOException; +import java.util.stream.IntStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.io.hfile.RowCacheKey; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({ MiscTests.class, MediumTests.class }) +public class TestBulkLoadHFilesRowCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBulkLoadHFilesRowCache.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();; + private static Admin admin; + + final static int NUM_CFS = 2; + final static byte[] QUAL = Bytes.toBytes("qual"); + final static int ROWCOUNT = 10; + + private TableName tableName; + private Table table; + private HRegion region; + + @Rule + public TestName testName = new TestName(); + + static String family(int i) { + return String.format("family_%04d", i); + } + + public static void buildHFiles(FileSystem fs, Path dir) throws IOException { + byte[] val = "value".getBytes(); + for (int i = 0; i < NUM_CFS; i++) { + Path testIn = new Path(dir, family(i)); + + TestHRegionServerBulkLoad.createHFile(fs, new Path(testIn, "hfile_" + i), + Bytes.toBytes(family(i)), QUAL, val, ROWCOUNT); + } + } + + private TableDescriptor createTableDesc(TableName name) { + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(name).setRowCacheEnabled(true); + IntStream.range(0, NUM_CFS).mapToObj(i -> ColumnFamilyDescriptorBuilder.of(family(i))) + .forEachOrdered(builder::setColumnFamily); + return builder.build(); + } + + private Path buildBulkFiles(TableName table) throws Exception { + Path dir = TEST_UTIL.getDataTestDirOnTestFS(table.getNameAsString()); + Path bulk1 = new Path(dir, table.getNameAsString()); + FileSystem fs = TEST_UTIL.getTestFileSystem(); + buildHFiles(fs, bulk1); + return bulk1; + } + + @BeforeClass + public static void setupCluster() throws Exception { + // To test simply, regardless of the number of HFiles + TEST_UTIL.getConfiguration().setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); + TEST_UTIL.startMiniCluster(1); + admin = TEST_UTIL.getAdmin(); + } + + @AfterClass + public static void teardownCluster() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Before + public void before() throws Exception { + tableName = TableName.valueOf(testName.getMethodName()); + admin.createTable(createTableDesc(tableName)); + table = TEST_UTIL.getConnection().getTable(tableName); + region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + } + + @After + public void after() throws Exception { + if (admin.tableExists(tableName)) { + admin.disableTable(tableName); + admin.deleteTable(tableName); + } + } + + @Test + public void testRowCache() throws Exception { + // Put a row to populate a row cache + Put put = new Put("row".getBytes()); + put.addColumn(family(0).getBytes(), "q1".getBytes(), "value".getBytes()); + table.put(put); + + // Ensure the region has a row cache + Get get = new Get("row".getBytes()); + table.get(get); + RowCacheKey keyPrev = new RowCacheKey(region, get.getRow()); + assertNotNull(region.getBlockCache().getBlock(keyPrev, true, false, true)); + + // bulkload + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf); + Path dir = buildBulkFiles(tableName); + loader.bulkLoad(tableName, dir); + + // Ensure the row cache is removed after bulkload + RowCacheKey keyCur = new RowCacheKey(region, get.getRow()); + assertNull(region.getBlockCache().getBlock(keyCur, true, false, true)); + // Ensure the row cache for keyPrev still exists, but it is not used any more. + assertNotNull(region.getBlockCache().getBlock(keyPrev, true, false, true)); + } +} From 80b728cf4fa93ac86c04f99680720d954db34de7 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Thu, 11 Sep 2025 11:50:26 +0900 Subject: [PATCH 02/21] Resolve compile error caused by ReturnValueIgnored --- .../hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java index 83ce32ee5511..d7bf0faf6b97 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java @@ -210,7 +210,7 @@ private static void testWith(Function func) { Assert.assertTrue(RowCacheService.canCacheRow(get, region)); // expect false - func.apply(get); + Get unused = func.apply(get); Assert.assertFalse(RowCacheService.canCacheRow(get, region)); } } From a6a8465da02eae443a8116bbe84aa08d62b7d467 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Tue, 23 Sep 2025 08:46:40 +0900 Subject: [PATCH 03/21] Separate RowCache from BlockCache - Implement RowCache - Initially considered modifying LruBlockCache, but the required changes were extensive. Instead, implemented RowCache using Caffeine cache. - Add row.cache.size configuration - Default is 0.0 (disabled); RowCache is enabled only if explicitly set to a value > 0. - The combined size of BlockCache + MemStore + RowCache must not exceed 80% of the heap. - Add Row Cache tab to RegionServer Block Cache UI - RowCache is not a BlockCache, but added here since there is no better place. - Add RowCache metrics - Metrics for size, count, eviction, hit, and miss are now exposed. --- .../org/apache/hadoop/hbase/HConstants.java | 22 ++-- .../hadoop/hbase/io/hfile/BlockType.java | 9 -- .../src/main/resources/hbase-default.xml | 6 + .../MetricsRegionServerSource.java | 9 +- .../MetricsRegionServerSourceImpl.java | 8 +- .../MetricsRegionServerWrapper.java | 12 +- .../tmpl/regionserver/BlockCacheTmpl.jamon | 58 +++++++++ .../tmpl/regionserver/RSStatusTmpl.jamon | 6 +- .../hadoop/hbase/io/hfile/CacheStats.java | 16 --- .../hadoop/hbase/io/util/MemorySizeUtil.java | 25 +++- .../hbase/regionserver/HRegionServer.java | 3 - .../MetricsRegionServerWrapperImpl.java | 32 ++++- .../hbase/regionserver/RSRpcServices.java | 5 + .../hadoop/hbase/regionserver/RowCache.java | 111 ++++++++++++++++ .../hfile => regionserver}/RowCacheKey.java | 29 ++--- .../hbase/regionserver/RowCacheService.java | 43 +++--- .../hadoop/hbase/regionserver/RowCells.java | 30 +---- .../io/hfile/TestCombinedBlockCache.java | 6 - .../hbase/io/util/TestMemorySizeUtil.java | 18 +++ .../MetricsRegionServerWrapperStub.java | 27 +++- .../regionserver/TestMetricsRegionServer.java | 7 +- .../hbase/regionserver/TestRowCache.java | 100 ++++++++------ .../regionserver/TestRowCacheCanCacheRow.java | 63 +++++++-- .../TestRowCacheClearRegionBlockCache.java | 122 ------------------ .../TestRowCacheConfigurationObserver.java | 20 ++- .../TestRowCacheKey.java | 3 +- .../regionserver/TestRowCacheService.java | 90 ++++++++++--- .../tool/TestBulkLoadHFilesRowCache.java | 22 +++- 28 files changed, 572 insertions(+), 330 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java rename hbase-server/src/main/java/org/apache/hadoop/hbase/{io/hfile => regionserver}/RowCacheKey.java (69%) delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java rename hbase-server/src/test/java/org/apache/hadoop/hbase/{io/hfile => regionserver}/TestRowCacheKey.java (97%) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 70241f6404db..997342bb6a8d 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1017,6 +1017,20 @@ public enum OperationStatusCode { public static final float HFILE_BLOCK_CACHE_SIZE_DEFAULT = 0.4f; + /** + * Configuration key for the size of the row cache + */ + public static final String ROW_CACHE_SIZE_KEY = "row.cache.size"; + public static final float ROW_CACHE_SIZE_DEFAULT = 0.0f; + + /** + * Configuration key for the minimum number of HFiles required to activate the Row Cache. If the + * number of HFiles is less than this value, the Row Cache does not operate even if it is enabled + * at the table level. + */ + public static final String ROW_CACHE_ACTIVATE_MIN_HFILES_KEY = "row.cache.activate.min.hfiles"; + public static final int ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT = 2; + /** * Configuration key for the memory size of the block cache */ @@ -1033,14 +1047,6 @@ public enum OperationStatusCode { public static final long HBASE_CLIENT_SCANNER_ONHEAP_BLOCK_CACHE_FIXED_SIZE_DEFAULT = 32 * 1024 * 1024L; - /** - * Configuration key for the minimum number of HFiles required to activate the Row Cache. If the - * number of HFiles is less than this value, the Row Cache does not operate even if it is enabled - * at the table level. - */ - public static final String ROW_CACHE_ACTIVATE_MIN_HFILES_KEY = "row.cache.activate.min.hfiles"; - public static final int ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT = 2; - /** * Configuration key for setting pread must read both necessaryLen and extraLen, default is * disabled. This is an optimized flag for reading HFile from blob storage. diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java index e73620e78bb5..dd1090833226 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java @@ -80,14 +80,6 @@ public int getId() { /** Fixed file trailer, both versions (always just a magic string) */ TRAILER("TRABLK\"$", BlockCategory.META), - // Pseudo block - - /** - * Cells of a row for row cache. This is a pseudo block type. It only exists to share the - * BlockCache interface. - */ - ROW_CELLS("ROWCELLS", BlockCategory.ROW), - // Legacy blocks /** Block index magic string in version 1 */ @@ -99,7 +91,6 @@ public enum BlockCategory { INDEX, BLOOM, ALL_CATEGORIES, - ROW, UNKNOWN; /** diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index d015ceb2d457..5e97b9541eba 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -1027,6 +1027,12 @@ possible configurations would overwhelm and obscure the important. This configuration allows setting an absolute memory size instead of a percentage of the maximum heap. Takes precedence over hfile.block.cache.size if both are specified. + + row.cache.size + 0.0 + Percentage of maximum heap (-Xmx setting) to allocate to row cache. + Default of 0.0; it means the row cache is disabled. + hfile.block.index.cacheonwrite false diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java index ad11e3143c91..166484fe8991 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java @@ -386,7 +386,6 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo String BLOCK_CACHE_GENERAL_BLOOM_META_MISS_COUNT = "blockCacheGeneralBloomMetaMissCount"; String BLOCK_CACHE_DELETE_FAMILY_BLOOM_MISS_COUNT = "blockCacheDeleteFamilyBloomMissCount"; String BLOCK_CACHE_TRAILER_MISS_COUNT = "blockCacheTrailerMissCount"; - String BLOCK_CACHE_ROW_MISS_COUNT = "blockCacheRowMissCount"; String BLOCK_CACHE_DATA_HIT_COUNT = "blockCacheDataHitCount"; String BLOCK_CACHE_ENCODED_DATA_HIT_COUNT = "blockCacheEncodedDataHitCount"; String BLOCK_CACHE_LEAF_INDEX_HIT_COUNT = "blockCacheLeafIndexHitCount"; @@ -398,7 +397,6 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo String BLOCK_CACHE_GENERAL_BLOOM_META_HIT_COUNT = "blockCacheGeneralBloomMetaHitCount"; String BLOCK_CACHE_DELETE_FAMILY_BLOOM_HIT_COUNT = "blockCacheDeleteFamilyBloomHitCount"; String BLOCK_CACHE_TRAILER_HIT_COUNT = "blockCacheTrailerHitCount"; - String BLOCK_CACHE_ROW_HIT_COUNT = "blockCacheRowHitCount"; String L1_CACHE_FREE_SIZE = "l1CacheFreeSize"; String L1_CACHE_FREE_SIZE_DESC = "Amount of free bytes in the L1 cache"; String L1_CACHE_SIZE = "l1CacheSize"; @@ -432,6 +430,13 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo String L2_CACHE_HIT_RATIO_DESC = "L2 cache hit ratio."; String L2_CACHE_MISS_RATIO = "l2CacheMissRatio"; String L2_CACHE_MISS_RATIO_DESC = "L2 cache miss ratio."; + + String ROW_CACHE_HIT_COUNT = "rowCacheHitCount"; + String ROW_CACHE_MISS_COUNT = "rowCacheMissCount"; + String ROW_CACHE_EVICTED_ROW_COUNT = "rowCacheEvictedRowCount"; + String ROW_CACHE_SIZE = "rowCacheSize"; + String ROW_CACHE_COUNT = "rowCacheCount"; + String RS_START_TIME_NAME = "regionServerStartTime"; String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum"; String SERVER_NAME_NAME = "serverName"; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java index 8fecb3c6f66d..90ea2a1165c8 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java @@ -436,7 +436,6 @@ public void getMetrics(MetricsCollector metricsCollector, boolean all) { .addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_MISS_COUNT, ""), rsWrap.getDeleteFamilyBloomMissCount()) .addCounter(Interns.info(BLOCK_CACHE_TRAILER_MISS_COUNT, ""), rsWrap.getTrailerMissCount()) - .addCounter(Interns.info(BLOCK_CACHE_ROW_MISS_COUNT, ""), rsWrap.getRowMissCount()) .addCounter(Interns.info(BLOCK_CACHE_DATA_HIT_COUNT, ""), rsWrap.getDataHitCount()) .addCounter(Interns.info(BLOCK_CACHE_LEAF_INDEX_HIT_COUNT, ""), rsWrap.getLeafIndexHitCount()) @@ -453,7 +452,12 @@ public void getMetrics(MetricsCollector metricsCollector, boolean all) { .addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_HIT_COUNT, ""), rsWrap.getDeleteFamilyBloomHitCount()) .addCounter(Interns.info(BLOCK_CACHE_TRAILER_HIT_COUNT, ""), rsWrap.getTrailerHitCount()) - .addCounter(Interns.info(BLOCK_CACHE_ROW_HIT_COUNT, ""), rsWrap.getRowHitCount()) + .addCounter(Interns.info(ROW_CACHE_HIT_COUNT, ""), rsWrap.getRowCacheHitCount()) + .addCounter(Interns.info(ROW_CACHE_MISS_COUNT, ""), rsWrap.getRowCacheMissCount()) + .addCounter(Interns.info(ROW_CACHE_EVICTED_ROW_COUNT, ""), + rsWrap.getRowCacheEvictedRowCount()) + .addGauge(Interns.info(ROW_CACHE_SIZE, ""), rsWrap.getRowCacheSize()) + .addGauge(Interns.info(ROW_CACHE_COUNT, ""), rsWrap.getRowCacheCount()) .addCounter(Interns.info(UPDATES_BLOCKED_TIME, UPDATES_BLOCKED_DESC), rsWrap.getUpdatesBlockedTime()) .addCounter(Interns.info(FLUSHED_CELLS, FLUSHED_CELLS_DESC), rsWrap.getFlushedCellsCount()) diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java index a665aec9b89f..68e43b276ee2 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java @@ -615,8 +615,6 @@ public interface MetricsRegionServerWrapper { long getTrailerMissCount(); - long getRowMissCount(); - long getDataHitCount(); long getLeafIndexHitCount(); @@ -637,7 +635,15 @@ public interface MetricsRegionServerWrapper { long getTrailerHitCount(); - long getRowHitCount(); + long getRowCacheHitCount(); + + long getRowCacheMissCount(); + + long getRowCacheSize(); + + long getRowCacheCount(); + + long getRowCacheEvictedRowCount(); long getTotalRowActionRequestCount(); diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon index 82609aad7190..b949c1025af3 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon @@ -21,6 +21,7 @@ Template for rendering Block Cache tabs in RegionServer Status page. CacheConfig cacheConfig; Configuration config; BlockCache bc; +RowCache rowCache; <%java> String bcUrl = bc == null ? null : "http://hbase.apache.org/devapidocs/" + bc.getClass().getName().replaceAll("\\.", "/") + ".html"; @@ -42,6 +43,7 @@ org.apache.hadoop.hbase.io.hfile.bucket.BucketCacheStats; org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator; org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket; +org.apache.hadoop.hbase.regionserver.RowCache; org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
@@ -51,6 +53,7 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; +
@@ -68,6 +71,9 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
<& bc_l; bc = bcs == null? null: bcs.length <= 1? null: bcs[1]; name = "L2"; evictions = evictions; &>
+
+ <& row_cache_stats; rowCache = rowCache &> +
@@ -562,3 +568,55 @@ Increase that value to get a complete picture.

+<%def row_cache_stats> +<%args> + RowCache rowCache; + +<%if rowCache == null %> +

RowCache is null

+<%else> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
AttributeValueDescription
Size<% TraditionalBinaryPrefix.long2String(rowCache.getSize(), "B", 1) %>Current size of row cache in use
Free<% TraditionalBinaryPrefix.long2String(rowCache.getMaxSize() - rowCache.getSize(), "B", 1) %>The total free memory currently available to store more cache entries
Count<% String.format("%,d", rowCache.getCount()) %>The number of rows in row cache
Evicted Rows<% String.format("%,d", rowCache.getEvictedRowCount()) %>The total number of rows evicted
Hits<% String.format("%,d", rowCache.getHitCount()) %>The number requests that were cache hits
Misses<% String.format("%,d", rowCache.getMissCount()) %>The number requests that were cache misses
All Time Hit Ratio<% String.format("%,.2f", rowCache.getHitCount() * 100.0 / (rowCache.getMissCount() + rowCache.getHitCount())) %><% "%" %>Hit Count divided by total requests count
+

RowCache is a separate cache distinct from BlockCache. Since there is no more appropriate place to display RowCache information, it is shown here alongside BlockCache.

+ + diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon index 4b8047b9e446..3351ddee3ef5 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon @@ -26,10 +26,12 @@ String bcv = ""; <%import> java.util.*; org.apache.hadoop.hbase.regionserver.HRegionServer; +org.apache.hadoop.hbase.regionserver.RowCacheService; org.apache.hadoop.hbase.client.RegionInfo; org.apache.hadoop.hbase.ServerName; org.apache.hadoop.hbase.HBaseConfiguration; org.apache.hadoop.hbase.io.hfile.CacheConfig; +org.apache.hadoop.hbase.regionserver.RowCache; org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo; org.apache.hadoop.hbase.util.JvmVersion; @@ -51,6 +53,8 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; ServerName masterServerName = masterAddressTracker == null ? null : masterAddressTracker.getMasterAddress(); int infoPort = masterAddressTracker == null ? 0 : masterAddressTracker.getMasterInfoPort(); + RowCacheService rowCacheService = regionServer.getRSRpcServices().getRowCacheService(); + RowCache rowCache = rowCacheService == null ? null : rowCacheService.getRowCache(); <%class> @@ -149,7 +153,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;

Block Cache

- <& BlockCacheTmpl; cacheConfig = new CacheConfig(regionServer.getConfiguration()); config = regionServer.getConfiguration(); bc = regionServer.getBlockCache().orElse(null) &> + <& BlockCacheTmpl; cacheConfig = new CacheConfig(regionServer.getConfiguration()); config = regionServer.getConfiguration(); bc = regionServer.getBlockCache().orElse(null); rowCache = rowCache &>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java index 9875b0c23362..fbf3e1087019 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java @@ -91,7 +91,6 @@ public class CacheStats { private final LongAdder generalBloomMetaMissCount = new LongAdder(); private final LongAdder deleteFamilyBloomMissCount = new LongAdder(); private final LongAdder trailerMissCount = new LongAdder(); - private final LongAdder rowMissCount = new LongAdder(); private final LongAdder dataHitCount = new LongAdder(); private final LongAdder leafIndexHitCount = new LongAdder(); @@ -103,7 +102,6 @@ public class CacheStats { private final LongAdder generalBloomMetaHitCount = new LongAdder(); private final LongAdder deleteFamilyBloomHitCount = new LongAdder(); private final LongAdder trailerHitCount = new LongAdder(); - private final LongAdder rowHitCount = new LongAdder(); // Executor for periodic cache stats rolling private ScheduledExecutorService metricsRollerScheduler; @@ -221,9 +219,6 @@ public void miss(boolean caching, boolean primary, BlockType type) { case TRAILER: trailerMissCount.increment(); break; - case ROW_CELLS: - rowMissCount.increment(); - break; default: // If there's a new type that's fine // Ignore it for now. This is metrics don't exception. @@ -271,9 +266,6 @@ public void hit(boolean caching, boolean primary, BlockType type) { case TRAILER: trailerHitCount.increment(); break; - case ROW_CELLS: - rowHitCount.increment(); - break; default: // If there's a new type that's fine // Ignore it for now. This is metrics don't exception. @@ -384,14 +376,6 @@ public long getTrailerHitCount() { return trailerHitCount.sum(); } - public long getRowHitCount() { - return rowHitCount.sum(); - } - - public long getRowMissCount() { - return rowMissCount.sum(); - } - public long getRequestCount() { return getHitCount() + getMissCount(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java index 7ada303d2939..6ad962a46348 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java @@ -93,25 +93,29 @@ public static void validateRegionServerHeapMemoryAllocation(Configuration conf) } float memStoreFraction = getGlobalMemStoreHeapPercent(conf, false); float blockCacheFraction = getBlockCacheHeapPercent(conf); + float rowCacheFraction = + conf.getFloat(HConstants.ROW_CACHE_SIZE_KEY, HConstants.ROW_CACHE_SIZE_DEFAULT); float minFreeHeapFraction = getRegionServerMinFreeHeapFraction(conf); int memStorePercent = (int) (memStoreFraction * 100); int blockCachePercent = (int) (blockCacheFraction * 100); + int rowCachePercent = (int) (rowCacheFraction * 100); int minFreeHeapPercent = (int) (minFreeHeapFraction * 100); - int usedPercent = memStorePercent + blockCachePercent; + int usedPercent = memStorePercent + blockCachePercent + rowCachePercent; int maxAllowedUsed = 100 - minFreeHeapPercent; if (usedPercent > maxAllowedUsed) { throw new RuntimeException(String.format( "RegionServer heap memory allocation is invalid: total memory usage exceeds 100%% " - + "(memStore + blockCache + requiredFreeHeap). " - + "Check the following configuration values:%n" + " - %s = %.2f%n" + " - %s = %s%n" - + " - %s = %s%n" + " - %s = %s", + + "(memStore + blockCache + rowCache + requiredFreeHeap). " + + "Check the following configuration values:" + "%n - %s = %.2f" + "%n - %s = %s" + + "%n - %s = %s" + "%n - %s = %s" + "%n - %s = %s", MEMSTORE_SIZE_KEY, memStoreFraction, HConstants.HFILE_BLOCK_CACHE_MEMORY_SIZE_KEY, conf.get(HConstants.HFILE_BLOCK_CACHE_MEMORY_SIZE_KEY), HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, conf.get(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY), HBASE_REGION_SERVER_FREE_HEAP_MIN_MEMORY_SIZE_KEY, - conf.get(HBASE_REGION_SERVER_FREE_HEAP_MIN_MEMORY_SIZE_KEY))); + conf.get(HBASE_REGION_SERVER_FREE_HEAP_MIN_MEMORY_SIZE_KEY), HConstants.ROW_CACHE_SIZE_KEY, + conf.get(HConstants.ROW_CACHE_SIZE_KEY))); } } @@ -313,4 +317,15 @@ public static long getBucketCacheSize(final Configuration conf) { } return (long) (bucketCacheSize * 1024 * 1024); } + + public static long getRowCacheSize(Configuration conf) { + long max = -1L; + final MemoryUsage usage = safeGetHeapMemoryUsage(); + if (usage != null) { + max = usage.getMax(); + } + float globalRowCachePercent = + conf.getFloat(HConstants.ROW_CACHE_SIZE_KEY, HConstants.ROW_CACHE_SIZE_DEFAULT); + return ((long) (max * globalRowCachePercent)); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 6ab78b14fbab..350baca36f46 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -3408,9 +3408,6 @@ CacheEvictionStats clearRegionBlockCache(Region region) { } } - // evict the entire row cache - evictedBlocks += blockCache.evictBlocksByHfileName(region.getRegionInfo().getEncodedName()); - return CacheEvictionStats.builder().withEvictedBlocks(evictedBlocks).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index 3732d492794e..45651624e63c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -68,6 +68,7 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { private BlockCache l1Cache = null; private BlockCache l2Cache = null; private MobFileCache mobFileCache; + private RowCache rowCache; private CacheStats cacheStats; private CacheStats l1Stats = null; private CacheStats l2Stats = null; @@ -99,6 +100,8 @@ public MetricsRegionServerWrapperImpl(final HRegionServer regionServer) { this.regionServer = regionServer; initBlockCache(); initMobFileCache(); + RSRpcServices rsRpcServices = this.regionServer.getRSRpcServices(); + this.rowCache = rsRpcServices == null ? null : rsRpcServices.getRowCacheService().getRowCache(); this.excludeDatanodeManager = this.regionServer.getWalFactory().getExcludeDatanodeManager(); this.period = regionServer.getConfiguration().getLong(HConstants.REGIONSERVER_METRICS_PERIOD, @@ -1144,11 +1147,6 @@ public long getTrailerMissCount() { return this.cacheStats != null ? this.cacheStats.getTrailerMissCount() : 0L; } - @Override - public long getRowMissCount() { - return this.cacheStats != null ? this.cacheStats.getRowMissCount() : 0L; - } - @Override public long getDataHitCount() { return this.cacheStats != null ? this.cacheStats.getDataHitCount() : 0L; @@ -1200,8 +1198,28 @@ public long getTrailerHitCount() { } @Override - public long getRowHitCount() { - return this.cacheStats != null ? this.cacheStats.getRowHitCount() : 0L; + public long getRowCacheHitCount() { + return this.rowCache != null ? this.rowCache.getHitCount() : 0L; + } + + @Override + public long getRowCacheMissCount() { + return this.rowCache != null ? this.rowCache.getMissCount() : 0L; + } + + @Override + public long getRowCacheSize() { + return this.rowCache != null ? this.rowCache.getSize() : 0L; + } + + @Override + public long getRowCacheCount() { + return this.rowCache != null ? this.rowCache.getCount() : 0L; + } + + @Override + public long getRowCacheEvictedRowCount() { + return this.rowCache != null ? this.rowCache.getEvictedRowCount() : 0L; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 0a84afe046f1..bc777c67df15 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -4077,4 +4077,9 @@ RegionScannerContext checkQuotaAndGetRegionScannerContext(ScanRequest request, Pair pair = newRegionScanner(request, region, builder); return new RegionScannerContext(pair.getFirst(), pair.getSecond(), quota); } + + // For testing only + public RowCacheService getRowCacheService() { + return rowCacheService; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java new file mode 100644 index 000000000000..d5b1cf3ae002 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.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.regionserver; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Policy; +import com.github.benmanes.caffeine.cache.RemovalCause; +import com.github.benmanes.caffeine.cache.RemovalListener; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.atomic.LongAdder; +import org.checkerframework.checker.nullness.qual.NonNull; + +/** + * A cache that stores rows retrieved by Get operations, using Caffeine as the underlying cache + * implementation. + */ +@org.apache.yetus.audience.InterfaceAudience.Private +public class RowCache { + private final class EvictionListener + implements RemovalListener<@NonNull RowCacheKey, @NonNull RowCells> { + @Override + public void onRemoval(RowCacheKey key, RowCells value, @NonNull RemovalCause cause) { + evictedRowCount.increment(); + } + } + + private final Cache<@NonNull RowCacheKey, RowCells> cache; + + private final LongAdder hitCount = new LongAdder(); + private final LongAdder missCount = new LongAdder(); + private final LongAdder evictedRowCount = new LongAdder(); + + RowCache(long maxSizeBytes) { + if (maxSizeBytes <= 0) { + cache = Caffeine.newBuilder().maximumSize(0).build(); + return; + } + + cache = + Caffeine.newBuilder().maximumWeight(maxSizeBytes).removalListener(new EvictionListener()) + .weigher((RowCacheKey key, + RowCells value) -> (int) Math.min(key.heapSize() + value.heapSize(), Integer.MAX_VALUE)) + .build(); + } + + void cacheBlock(RowCacheKey key, RowCells value) { + cache.put(key, value); + } + + public RowCells getBlock(RowCacheKey key, boolean caching) { + if (!caching) { + missCount.increment(); + return null; + } + + RowCells value = cache.getIfPresent(key); + if (value == null) { + missCount.increment(); + } else { + hitCount.increment(); + } + return value; + } + + void evictBlock(RowCacheKey key) { + cache.asMap().remove(key); + } + + public long getHitCount() { + return hitCount.sum(); + } + + public long getMissCount() { + return missCount.sum(); + } + + public long getEvictedRowCount() { + return evictedRowCount.sum(); + } + + public long getSize() { + Optional result = cache.policy().eviction().map(Policy.Eviction::weightedSize); + return result.orElse(OptionalLong.of(-1L)).orElse(-1L); + } + + public long getMaxSize() { + Optional result = cache.policy().eviction().map(Policy.Eviction::getMaximum); + return result.orElse(-1L); + } + + public long getCount() { + return cache.estimatedSize(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/RowCacheKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheKey.java similarity index 69% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/RowCacheKey.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheKey.java index d26af3213c1f..dc293ec8a053 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/RowCacheKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheKey.java @@ -15,54 +15,49 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.io.hfile; +package org.apache.hadoop.hbase.regionserver; import java.util.Arrays; import java.util.Objects; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.yetus.audience.InterfaceAudience; -/** - * Cache Key for use with implementations of {@link BlockCache} - */ -@InterfaceAudience.Private -public class RowCacheKey extends BlockCacheKey { - private static final long serialVersionUID = -686874540957524887L; +@org.apache.yetus.audience.InterfaceAudience.Private +public class RowCacheKey implements HeapSize { public static final long FIXED_OVERHEAD = ClassSize.estimateBase(RowCacheKey.class, false); + private final String encodedRegionName; private final byte[] rowKey; + // Row cache keys should not be evicted on close, since the cache may contain many entries and // eviction would be slow. Instead, the region’s rowCacheSeqNum is used to generate new keys that // ignore the existing cache when the region is reopened or bulk-loaded. private final long rowCacheSeqNum; public RowCacheKey(HRegion region, byte[] rowKey) { - super(region.getRegionInfo().getEncodedName(), 0, region.getRegionInfo().getReplicaId() == 0, - BlockType.ROW_CELLS); - + this.encodedRegionName = region.getRegionInfo().getEncodedName(); this.rowKey = Objects.requireNonNull(rowKey, "rowKey cannot be null"); this.rowCacheSeqNum = region.getRowCacheSeqNum(); } @Override public boolean equals(Object o) { - if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; RowCacheKey that = (RowCacheKey) o; - return rowCacheSeqNum == that.rowCacheSeqNum && Arrays.equals(rowKey, that.rowKey); + return rowCacheSeqNum == that.rowCacheSeqNum + && Objects.equals(encodedRegionName, that.encodedRegionName) + && Objects.deepEquals(rowKey, that.rowKey); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), Arrays.hashCode(rowKey), Long.hashCode(rowCacheSeqNum)); + return Objects.hash(encodedRegionName, Arrays.hashCode(rowKey), rowCacheSeqNum); } @Override public String toString() { - return super.toString() + '_' + Bytes.toStringBinary(this.rowKey) + '_' + rowCacheSeqNum; + return encodedRegionName + '_' + Bytes.toStringBinary(rowKey) + '_' + rowCacheSeqNum; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index 636961ae5f1b..a485e19538c8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -17,9 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT; -import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; - import java.io.IOException; import java.util.Arrays; import java.util.HashSet; @@ -32,6 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.CheckAndMutate; import org.apache.hadoop.hbase.client.CheckAndMutateResult; @@ -41,7 +39,7 @@ import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.io.hfile.RowCacheKey; +import org.apache.hadoop.hbase.io.util.MemorySizeUtil; import org.apache.hadoop.hbase.ipc.RpcCallContext; import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement; import org.apache.hadoop.hbase.quotas.OperationQuota; @@ -55,7 +53,8 @@ /** * It is responsible for populating the row cache and retrieving rows from it. */ -class RowCacheService { +@org.apache.yetus.audience.InterfaceAudience.Private +public class RowCacheService { /** * A barrier that prevents the row cache from being populated during table operations, such as * bulk loads. It is implemented as a counter to address issues that arise when the same table is @@ -68,6 +67,10 @@ class RowCacheService { * concurrently. */ private final Map rowLevelBarrierMap = new ConcurrentHashMap<>(); + + private final boolean enabledByConf; + private final RowCache rowCache; + private int activateMinHFiles; @FunctionalInterface @@ -77,11 +80,15 @@ interface RowOperation { RowCacheService(Configuration conf) { updateConf(conf); + + enabledByConf = + conf.getFloat(HConstants.ROW_CACHE_SIZE_KEY, HConstants.ROW_CACHE_SIZE_DEFAULT) > 0; + rowCache = new RowCache(MemorySizeUtil.getRowCacheSize(conf)); } synchronized void updateConf(Configuration conf) { - this.activateMinHFiles = - conf.getInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT); + this.activateMinHFiles = conf.getInt(HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, + HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT); } RegionScannerImpl getScanner(HRegion region, Get get, Scan scan, List results) @@ -117,8 +124,7 @@ private RegionScannerImpl getScannerInternal(HRegion region, Scan scan, List results) { - RowCells row = - (RowCells) region.getBlockCache().getBlock(key, get.getCacheBlocks(), false, true); + RowCells row = rowCache.getBlock(key, get.getCacheBlocks()); if (row == null) { return false; @@ -143,7 +149,7 @@ private void populateCache(HRegion region, List results, RowCacheKey key) // The row cache is populated only when no row level barriers remain rowLevelBarrierMap.computeIfAbsent(key, k -> { try { - region.getBlockCache().cacheBlock(key, new RowCells(results), false); + rowCache.cacheBlock(key, new RowCells(results)); } catch (CloneNotSupportedException ignored) { // Not able to cache row cells, ignore } @@ -223,8 +229,8 @@ void createTableLevelBarrier(TableName tableName) { * @return true if the row can be cached, false otherwise */ // @formatter:on - static boolean canCacheRow(Get get, Region region) { - return region.getTableDescriptor().isRowCacheEnabled() && get.getCacheBlocks() + boolean canCacheRow(Get get, Region region) { + return enabledByConf && region.getTableDescriptor().isRowCacheEnabled() && get.getCacheBlocks() && get.getFilter() == null && isRetrieveAllCells(get, region) && isDefaultTtl(region) && get.getAttributesMap().isEmpty() && !get.isCheckExistenceOnly() && get.getColumnFamilyTimeRange().isEmpty() && get.getConsistency() == Consistency.STRONG @@ -263,7 +269,7 @@ private R mutateWithRowCacheBarrier(HRegion region, List mutations // After creating the barrier, evict the existing row cache for this row, // as it becomes invalid after the mutation - evictRowCache(region, key); + evictRowCache(key); }); return execute(operation); @@ -287,7 +293,7 @@ private R mutateWithRowCacheBarrier(HRegion region, byte[] row, RowOperation // After creating the barrier, evict the existing row cache for this row, // as it becomes invalid after the mutation - evictRowCache(region, key); + evictRowCache(key); return execute(operation); } finally { @@ -300,8 +306,8 @@ R execute(RowOperation operation) throws IOException { return operation.execute(); } - void evictRowCache(HRegion region, RowCacheKey key) { - region.getBlockCache().evictBlock(key); + void evictRowCache(RowCacheKey key) { + rowCache.evictBlock(key); } /** @@ -358,4 +364,9 @@ AtomicInteger getRowLevelBarrier(RowCacheKey key) { AtomicInteger getTableLevelBarrier(TableName tableName) { return tableLevelBarrierMap.get(tableName); } + + // For testing only + public RowCache getRowCache() { + return rowCache; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java index 59e3271bcb2d..af0a0ea4c537 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCells.java @@ -17,17 +17,15 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.ExtendedCell; -import org.apache.hadoop.hbase.io.hfile.BlockType; -import org.apache.hadoop.hbase.io.hfile.Cacheable; -import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer; +import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.util.ClassSize; -class RowCells implements Cacheable { +@org.apache.yetus.audience.InterfaceAudience.Private +public class RowCells implements HeapSize { public static final long FIXED_OVERHEAD = ClassSize.estimateBase(RowCells.class, false); private final List cells = new ArrayList<>(); @@ -46,28 +44,6 @@ public RowCells(List cells) throws CloneNotSupportedException { } } - @Override - public int getSerializedLength() { - // Cannot be serialized. Cached on heap only and cannot be moved off heap. - return 0; - } - - @Override - public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) { - // Cannot be serialized. Cached on heap only and cannot be moved off heap. - } - - @Override - public CacheableDeserializer getDeserializer() { - // Cannot be serialized. Cached on heap only and cannot be moved off heap. - return null; - } - - @Override - public BlockType getBlockType() { - return BlockType.ROW_CELLS; - } - @Override public long heapSize() { long cellsSize = cells.stream().mapToLong(Cell::heapSize).sum(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java index 49558b4b72a4..b9bca1ba6b4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java @@ -133,12 +133,6 @@ public void testCombinedBlockCacheStatsWithNoBlockType() throws Exception { testCombinedBlockCacheStats(null, 0, 1); } - @Test - public void testCombinedBlockCacheStatsWithRowCellsBlockType() throws Exception { - // ROW_CELLS type is cached only in the L1 cache, since it is not a DATA block type - testCombinedBlockCacheStats(BlockType.ROW_CELLS, 1, 0); - } - private CombinedBlockCache createCombinedBlockCache() { Configuration conf = UTIL.getConfiguration(); conf.set(BUCKET_CACHE_IOENGINE_KEY, "offheap"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/util/TestMemorySizeUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/util/TestMemorySizeUtil.java index 5f00c34dbcb0..3cf1fc33753a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/util/TestMemorySizeUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/util/TestMemorySizeUtil.java @@ -52,6 +52,13 @@ public void testValidateRegionServerHeapMemoryAllocation() { assertEquals(HConstants.HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD, 0.2f, 0.0f); MemorySizeUtil.validateRegionServerHeapMemoryAllocation(conf); + // when memstore size + block cache size + row cache size + default free heap min size == 1.0 + conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.4f); + conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.39f); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); + assertEquals(HConstants.HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD, 0.2f, 0.0f); + MemorySizeUtil.validateRegionServerHeapMemoryAllocation(conf); + // when memstore size + block cache size + default free heap min size > 1.0 conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.5f); assertThrows(RuntimeException.class, @@ -60,6 +67,7 @@ public void testValidateRegionServerHeapMemoryAllocation() { // when free heap min size is set to 0, it should not throw an exception conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.5f); conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.5f); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.0f); conf.setLong(MemorySizeUtil.HBASE_REGION_SERVER_FREE_HEAP_MIN_MEMORY_SIZE_KEY, 0L); MemorySizeUtil.validateRegionServerHeapMemoryAllocation(conf); @@ -86,4 +94,14 @@ public void testGetRegionServerMinFreeHeapFraction() { minFreeHeapFraction = MemorySizeUtil.getRegionServerMinFreeHeapFraction(conf); assertEquals(0.0f, minFreeHeapFraction, 0.0f); } + + @Test + public void testGetRowCacheSize() { + float rowCacheSizeRatio = 0.01f; + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, rowCacheSizeRatio); + long rowCacheSizeBytes = MemorySizeUtil.getRowCacheSize(conf); + + long maxMemory = MemorySizeUtil.safeGetHeapMemoryUsage().getMax(); + assertEquals((long) (maxMemory * rowCacheSizeRatio), rowCacheSizeBytes); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java index 5dad6e21e5c9..6b677f2d1223 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java @@ -612,11 +612,6 @@ public long getTrailerMissCount() { return 0; } - @Override - public long getRowMissCount() { - return 1; - } - @Override public long getDataHitCount() { return 0; @@ -668,7 +663,27 @@ public long getTrailerHitCount() { } @Override - public long getRowHitCount() { + public long getRowCacheHitCount() { + return 2; + } + + @Override + public long getRowCacheMissCount() { + return 1; + } + + @Override + public long getRowCacheEvictedRowCount() { + return 0; + } + + @Override + public long getRowCacheSize() { + return 1; + } + + @Override + public long getRowCacheCount() { return 2; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java index 39ec4065769c..76c2a8ad6e42 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java @@ -124,10 +124,8 @@ public void testWrapperSource() { HELPER.assertGauge("blockCacheDataBlockCount", 300, serverSource); HELPER.assertGauge("blockCacheSize", 415, serverSource); HELPER.assertCounter("blockCacheHitCount", 416, serverSource); - HELPER.assertCounter("blockCacheRowHitCount", 2, serverSource); HELPER.assertCounter("blockCacheHitCachingCount", 16, serverSource); HELPER.assertCounter("blockCacheMissCount", 417, serverSource); - HELPER.assertCounter("blockCacheRowMissCount", 1, serverSource); HELPER.assertCounter("blockCacheMissCachingCount", 17, serverSource); HELPER.assertCounter("blockCacheEvictionCount", 418, serverSource); HELPER.assertGauge("blockCacheCountHitPercent", 98, serverSource); @@ -150,6 +148,11 @@ public void testWrapperSource() { HELPER.assertGauge("l2CacheHitRatio", 90, serverSource); HELPER.assertGauge("l2CacheMissRatio", 10, serverSource); HELPER.assertCounter("updatesBlockedTime", 419, serverSource); + HELPER.assertCounter("rowCacheHitCount", 2, serverSource); + HELPER.assertCounter("rowCacheMissCount", 1, serverSource); + HELPER.assertCounter("rowCacheEvictedRowCount", 0, serverSource); + HELPER.assertGauge("rowCacheSize", 1, serverSource); + HELPER.assertGauge("rowCacheCount", 2, serverSource); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java index ce6545ea6c3e..16443c4a95ea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java @@ -17,7 +17,12 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; +import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.ROW_CACHE_EVICTED_ROW_COUNT; +import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.ROW_CACHE_HIT_COUNT; +import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.ROW_CACHE_MISS_COUNT; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -53,7 +58,6 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; -import org.apache.hadoop.hbase.io.hfile.RowCacheKey; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -85,6 +89,7 @@ public class TestRowCache { private static MetricsRegionServerSource serverSource; private static Admin admin; + private static RowCache rowCache; private TableName tableName; private Table table; @@ -96,6 +101,11 @@ public class TestRowCache { @BeforeClass public static void beforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); + + // Enable row cache but reduce the block cache size to fit in 80% of the heap + conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); + conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); + // To test simply, regardless of the number of HFiles conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); @@ -105,6 +115,9 @@ public static void beforeClass() throws Exception { metricsHelper = CompatibilityFactory.getInstance(MetricsAssertHelper.class); metricsRegionServer = cluster.getRegionServer(0).getMetrics(); serverSource = metricsRegionServer.getMetricsSource(); + + rowCache = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRSRpcServices() + .getRowCacheService().getRowCache(); } @AfterClass @@ -175,8 +188,12 @@ public void testGetWithRowCache() throws IOException, InterruptedException { // Initialize metrics recomputeMetrics(); setCounterBase("Get_num_ops", metricsHelper.getCounter("Get_num_ops", serverSource)); - setCounterBase("blockCacheRowHitCount", - metricsHelper.getCounter("blockCacheRowHitCount", serverSource)); + setCounterBase(ROW_CACHE_HIT_COUNT, + metricsHelper.getCounter(ROW_CACHE_HIT_COUNT, serverSource)); + setCounterBase(ROW_CACHE_MISS_COUNT, + metricsHelper.getCounter(ROW_CACHE_MISS_COUNT, serverSource)); + setCounterBase(ROW_CACHE_EVICTED_ROW_COUNT, + metricsHelper.getCounter(ROW_CACHE_EVICTED_ROW_COUNT, serverSource)); // First get to populate the row cache result = table.get(get); @@ -188,7 +205,8 @@ public void testGetWithRowCache() throws IOException, InterruptedException { assertArrayEquals("22".getBytes(), result.getValue(CF2, Q2)); assertCounterDiff("Get_num_ops", 1); // Ensure the get operation from HFile without row cache - assertCounterDiff("blockCacheRowHitCount", 0); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); + assertCounterDiff(ROW_CACHE_MISS_COUNT, 1); // Get from the row cache result = table.get(get); @@ -200,12 +218,16 @@ public void testGetWithRowCache() throws IOException, InterruptedException { assertArrayEquals("22".getBytes(), result.getValue(CF2, Q2)); assertCounterDiff("Get_num_ops", 1); // Ensure the get operation from the row cache - assertCounterDiff("blockCacheRowHitCount", 1); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 1); + assertCounterDiff(ROW_CACHE_MISS_COUNT, 0); // Row cache is invalidated by the put operation - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); table.put(put); - assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + recomputeMetrics(); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 1); + assertCounterDiff(ROW_CACHE_MISS_COUNT, 0); + assertCounterDiff(ROW_CACHE_EVICTED_ROW_COUNT, 1); // Get is executed without the row cache; however, the cache is re-populated as a result result = table.get(get); @@ -213,7 +235,9 @@ public void testGetWithRowCache() throws IOException, InterruptedException { assertArrayEquals(rowKey, result.getRow()); assertCounterDiff("Get_num_ops", 1); // Ensure the get operation not from the row cache - assertCounterDiff("blockCacheRowHitCount", 0); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); + assertCounterDiff(ROW_CACHE_MISS_COUNT, 1); + assertCounterDiff(ROW_CACHE_EVICTED_ROW_COUNT, 0); // Get again with the row cache result = table.get(get); @@ -221,34 +245,36 @@ public void testGetWithRowCache() throws IOException, InterruptedException { assertArrayEquals(rowKey, result.getRow()); assertCounterDiff("Get_num_ops", 1); // Ensure the get operation from the row cache - assertCounterDiff("blockCacheRowHitCount", 1); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 1); + assertCounterDiff(ROW_CACHE_MISS_COUNT, 0); + assertCounterDiff(ROW_CACHE_EVICTED_ROW_COUNT, 0); // Row cache is invalidated by the increment operation - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); table.incrementColumnValue(rowKey, CF1, Q1, 1); - assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey, true)); // Get is executed without the row cache; however, the cache is re-populated as a result table.get(get); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); // Row cache is invalidated by the append operation - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); Append append = new Append(rowKey); append.addColumn(CF1, Q1, Bytes.toBytes(0L)); table.append(append); - assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey, true)); // Get is executed without the row cache; however, the cache is re-populated as a result table.get(get); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); // Row cache is invalidated by the delete operation - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); Delete delete = new Delete(rowKey); delete.addColumn(CF1, Q1); table.delete(delete); - assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey, true)); } @Test(expected = DoNotRetryIOException.class) @@ -281,7 +307,7 @@ public void testCheckAndMutate() throws IOException, InterruptedException { // Validate that the row cache is populated result = table.get(get); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); @@ -291,11 +317,11 @@ public void testCheckAndMutate() throws IOException, InterruptedException { cam = CheckAndMutate.newBuilder(rowKey).ifEquals(CF1, Q2, "00".getBytes()).build(put2); camResult = table.checkAndMutate(cam); assertFalse(camResult.isSuccess()); - assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey, true)); // Validate that the row cache is populated result = table.get(get); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); @@ -303,7 +329,7 @@ public void testCheckAndMutate() throws IOException, InterruptedException { cam = CheckAndMutate.newBuilder(rowKey).ifEquals(CF1, Q2, "12".getBytes()).build(put2); camResult = table.checkAndMutate(cam); assertTrue(camResult.isSuccess()); - assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey, true)); } @Test @@ -333,11 +359,11 @@ public void testCheckAndMutates() throws IOException, InterruptedException { // Validate that the row caches are populated result1 = table.get(get1); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey1, true)); assertArrayEquals("111".getBytes(), result1.getValue(CF1, Q1)); assertArrayEquals("112".getBytes(), result1.getValue(CF1, Q2)); result2 = table.get(get2); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey2, true)); assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); @@ -348,8 +374,8 @@ public void testCheckAndMutates() throws IOException, InterruptedException { camResults = table.checkAndMutate(cams); assertTrue(camResults.get(0).isSuccess()); assertTrue(camResults.get(1).isSuccess()); - assertNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); - assertNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey1, true)); + assertNull(rowCache.getBlock(rowCacheKey2, true)); } @Test @@ -377,11 +403,11 @@ public void testRowMutations() throws IOException, InterruptedException { // Validate that the row caches are populated result1 = table.get(get1); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey1, true)); assertArrayEquals("111".getBytes(), result1.getValue(CF1, Q1)); assertArrayEquals("112".getBytes(), result1.getValue(CF1, Q2)); result2 = table.get(get2); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey1, true)); assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); @@ -396,16 +422,16 @@ public void testRowMutations() throws IOException, InterruptedException { CheckAndMutate cam = CheckAndMutate.newBuilder(rowKey1).ifEquals(CF1, Q1, "111".getBytes()).build(rms); table.checkAndMutate(cam); - assertNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey1, true)); + assertNotNull(rowCache.getBlock(rowCacheKey2, true)); // Validate that the row caches are populated result1 = table.get(get1); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey1, true)); assertArrayEquals("111111".getBytes(), result1.getValue(CF1, Q1)); assertArrayEquals("112112".getBytes(), result1.getValue(CF1, Q2)); result2 = table.get(get2); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey1, true)); assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); } @@ -452,13 +478,13 @@ public void testBatch() throws IOException, InterruptedException { results = new Object[batchOperations.size()]; table.batch(batchOperations, results); assertEquals(3, results.length); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey1, true)); assertArrayEquals("111".getBytes(), ((Result) results[0]).getValue(CF1, Q1)); assertArrayEquals("112".getBytes(), ((Result) results[0]).getValue(CF1, Q2)); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey2, true)); assertArrayEquals("211".getBytes(), ((Result) results[1]).getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), ((Result) results[1]).getValue(CF1, Q2)); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey3, false, false, false)); + assertNotNull(rowCache.getBlock(rowCacheKey3, true)); assertArrayEquals("311".getBytes(), ((Result) results[2]).getValue(CF1, Q1)); assertArrayEquals("312".getBytes(), ((Result) results[2]).getValue(CF1, Q2)); @@ -474,8 +500,8 @@ public void testBatch() throws IOException, InterruptedException { results = new Object[batchOperations.size()]; table.batch(batchOperations, results); assertEquals(2, results.length); - assertNull(region.getBlockCache().getBlock(rowCacheKey1, false, false, false)); - assertNull(region.getBlockCache().getBlock(rowCacheKey2, false, false, false)); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey3, false, false, false)); + assertNull(rowCache.getBlock(rowCacheKey1, true)); + assertNull(rowCache.getBlock(rowCacheKey2, true)); + assertNotNull(rowCache.getBlock(rowCacheKey3, true)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java index d7bf0faf6b97..5c51b63c304b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java @@ -19,8 +19,11 @@ import java.io.IOException; import java.util.function.Function; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -52,7 +55,7 @@ public class TestRowCacheCanCacheRow { HBaseClassTestRule.forClass(TestRowCacheCanCacheRow.class); @Test - public void testRowCacheEnabled() { + public void testRowCacheEnabledByTable() { Region region = Mockito.mock(Region.class); ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); TableDescriptor td; @@ -63,13 +66,39 @@ public void testRowCacheEnabled() { td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true).setColumnFamily(cfd) .build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); - Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + Configuration conf = HBaseConfiguration.create(); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); + Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + + RowCacheService rowCacheService = new RowCacheService(conf); + Assert.assertTrue(rowCacheService.canCacheRow(get, region)); // Disable row cache, expect false td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(cfd) .setRowCacheEnabled(false).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); - Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + Assert.assertFalse(rowCacheService.canCacheRow(get, region)); + } + + @Test + public void testRowCacheDisabledByConfig() { + Region region = Mockito.mock(Region.class); + Configuration conf = HBaseConfiguration.create(); + Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + + ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + TableDescriptor td; + + Get get = new Get(ROW_KEY); + get.addFamily(CF1); + + // Row cache enabled at table level, but disabled by row cache size 0, expect false + td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true).setColumnFamily(cfd) + .build(); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + + RowCacheService rowCacheService = new RowCacheService(conf); + Assert.assertFalse(rowCacheService.canCacheRow(get, region)); } @Test @@ -80,19 +109,23 @@ public void testRetrieveAllCells() { TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); + Configuration conf = HBaseConfiguration.create(); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); + Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + RowCacheService rowCacheService = new RowCacheService(conf); // Not all CFs, expect false Get get = new Get(ROW_KEY); get.addFamily(CF1); - Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + Assert.assertFalse(rowCacheService.canCacheRow(get, region)); // All CFs, expect true get.addFamily(CF2); - Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + Assert.assertTrue(rowCacheService.canCacheRow(get, region)); // Not all qualifiers, expect false get.addColumn(CF1, "q1".getBytes()); - Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + Assert.assertFalse(rowCacheService.canCacheRow(get, region)); } @Test @@ -101,6 +134,11 @@ public void testTtl() { ColumnFamilyDescriptor cfd2; TableDescriptor td; Region region = Mockito.mock(Region.class); + Configuration conf = HBaseConfiguration.create(); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); + Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + RowCacheService rowCacheService = new RowCacheService(conf); + Get get = new Get(ROW_KEY); get.addFamily(CF1); get.addFamily(CF2); @@ -111,14 +149,14 @@ public void testTtl() { td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); - Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + Assert.assertFalse(rowCacheService.canCacheRow(get, region)); // Ttl is not set, expect true cfd1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); - Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + Assert.assertTrue(rowCacheService.canCacheRow(get, region)); } @Test @@ -205,12 +243,17 @@ private static void testWith(Function func) { .setColumnFamily(cfd).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); + Configuration conf = HBaseConfiguration.create(); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); + Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + RowCacheService rowCacheService = new RowCacheService(conf); + Get get = new Get(ROW_KEY); get.addFamily(CF1); - Assert.assertTrue(RowCacheService.canCacheRow(get, region)); + Assert.assertTrue(rowCacheService.canCacheRow(get, region)); // expect false Get unused = func.apply(get); - Assert.assertFalse(RowCacheService.canCacheRow(get, region)); + Assert.assertFalse(rowCacheService.canCacheRow(get, region)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java deleted file mode 100644 index 4b528d3b5e62..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheClearRegionBlockCache.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.regionserver; - -import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.io.hfile.RowCacheKey; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@Category(MediumTests.class) -@RunWith(Parameterized.class) -public class TestRowCacheClearRegionBlockCache { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestRowCacheClearRegionBlockCache.class); - - private static final byte[] FAMILY = Bytes.toBytes("family"); - private static final byte[][] SPLIT_KEY = new byte[][] { Bytes.toBytes("5") }; - private static final int NUM_RS = 2; - - private final HBaseTestingUtil HTU = new HBaseTestingUtil(); - - private Admin admin; - private Table table; - private TableName tableName; - - @Rule - public TestName testName = new TestName(); - - @Parameterized.Parameter - public String cacheType; - - @Parameterized.Parameters(name = "{index}: {0}") - public static Object[] data() { - return new Object[] { "lru", "bucket" }; - } - - @Before - public void setup() throws Exception { - if (cacheType.equals("bucket")) { - Configuration conf = HTU.getConfiguration(); - conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap"); - conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 30); - } - - // To test simply, regardless of the number of HFiles - HTU.getConfiguration().setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); - HTU.startMiniCluster(NUM_RS); - admin = HTU.getAdmin(); - - // Create table - tableName = TableName.valueOf(testName.getMethodName().replaceAll("[\\[\\]: ]", "_")); - ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build(); - TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(cfd) - .setRowCacheEnabled(true).build(); - admin.createTable(td, SPLIT_KEY); - table = admin.getConnection().getTable(tableName); - - HTU.loadNumericRows(table, FAMILY, 1, 10); - } - - @After - public void teardown() throws Exception { - HTU.shutdownMiniCluster(); - } - - @Test - public void testClearRowCache() throws Exception { - HRegion region = HTU.getRSForFirstRegionInTable(tableName).getRegions().stream() - .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); - - Get get = new Get(Bytes.toBytes("1")); - table.get(get); - - // Ensure a row cache entry exists - RowCacheKey rowCacheKey = new RowCacheKey(region, Bytes.toBytes("1")); - assertNotNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); - - // Ensure the row cache is cleared - admin.clearBlockCache(tableName); - assertNull(region.getBlockCache().getBlock(rowCacheKey, false, false, false)); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java index 2d29bc8796d4..d0dd09f855b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java @@ -17,7 +17,10 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; +import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.ROW_CACHE_HIT_COUNT; import static org.junit.Assert.assertArrayEquals; import java.io.IOException; @@ -79,6 +82,11 @@ public class TestRowCacheConfigurationObserver { @BeforeClass public static void beforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); + + // Enable row cache but reduce the block cache size to fit in 80% of the heap + conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); + conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); + // Set a value different from the default conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); @@ -154,8 +162,8 @@ public void testRowCacheWithHFilesCount() throws IOException { // Initialize metrics recomputeMetrics(); setCounterBase("Get_num_ops", metricsHelper.getCounter("Get_num_ops", serverSource)); - setCounterBase("blockCacheRowHitCount", - metricsHelper.getCounter("blockCacheRowHitCount", serverSource)); + setCounterBase(ROW_CACHE_HIT_COUNT, + metricsHelper.getCounter(ROW_CACHE_HIT_COUNT, serverSource)); Put put = new Put(rowKey); put.addColumn(CF1, Q1, "11".getBytes()); @@ -167,7 +175,7 @@ public void testRowCacheWithHFilesCount() throws IOException { assertArrayEquals(rowKey, result.getRow()); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertCounterDiff("Get_num_ops", 1); - assertCounterDiff("blockCacheRowHitCount", 0); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); // Flush, 1 store file exists TEST_UTIL.getAdmin().flush(tableName); @@ -178,7 +186,7 @@ public void testRowCacheWithHFilesCount() throws IOException { assertArrayEquals(rowKey, result.getRow()); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertCounterDiff("Get_num_ops", 1); - assertCounterDiff("blockCacheRowHitCount", 0); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); // Flush, 2(ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT) store files exist table.put(put); @@ -191,7 +199,7 @@ public void testRowCacheWithHFilesCount() throws IOException { assertArrayEquals(rowKey, result.getRow()); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertCounterDiff("Get_num_ops", 1); - assertCounterDiff("blockCacheRowHitCount", 0); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); // Get again, this time the row cache is hit result = table.get(get); @@ -199,6 +207,6 @@ public void testRowCacheWithHFilesCount() throws IOException { assertArrayEquals(rowKey, result.getRow()); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertCounterDiff("Get_num_ops", 1); - assertCounterDiff("blockCacheRowHitCount", 1); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 1); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowCacheKey.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheKey.java similarity index 97% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowCacheKey.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheKey.java index 355f5ac612c3..ee75fd251924 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowCacheKey.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheKey.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.io.hfile; +package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.BeforeClass; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java index 49908704fba4..f396f8bc248b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.CheckAndMutate; @@ -41,8 +42,6 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.io.hfile.BlockCache; -import org.apache.hadoop.hbase.io.hfile.LruBlockCache; -import org.apache.hadoop.hbase.io.hfile.RowCacheKey; import org.apache.hadoop.hbase.ipc.RpcCallContext; import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement; import org.apache.hadoop.hbase.quotas.OperationQuota; @@ -80,8 +79,6 @@ public void testBarrier() throws IOException { Mockito.when(hStore.getStorefilesCount()).thenReturn(2); stores.add(hStore); - BlockCache blockCache = new LruBlockCache(1024, 64 * 1024); - ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder("CF1".getBytes()).build(); TableDescriptor td = Mockito.mock(TableDescriptor.class); Mockito.when(td.isRowCacheEnabled()).thenReturn(true); @@ -93,29 +90,31 @@ public void testBarrier() throws IOException { RegionScannerImpl regionScanner = Mockito.mock(RegionScannerImpl.class); + Configuration conf = HBaseConfiguration.create(); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); HRegion region = Mockito.mock(HRegion.class); Mockito.when(region.getRegionInfo()).thenReturn(regionInfo); - Mockito.when(region.getBlockCache()).thenReturn(blockCache); Mockito.when(region.getTableDescriptor()).thenReturn(td); Mockito.when(region.getStores()).thenReturn(stores); Mockito.when(region.getScanner(scan)).thenReturn(regionScanner); + Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); RowCacheKey key = new RowCacheKey(region, rowKey); List results = new ArrayList<>(); results.add(KeyValueTestUtil.create("row", "CF", "q1", 1, "v1")); // Initialize RowCacheService - Configuration conf = HBaseConfiguration.create(); RowCacheService rowCacheService = new RowCacheService(conf); + RowCache rowCache = rowCacheService.getRowCache(); // Verify that row cache populated before creating a row level barrier rowCacheService.getScanner(region, get, scan, results); - assertNotNull(blockCache.getBlock(key, true, false, true)); + assertNotNull(rowCache.getBlock(key, true)); assertNull(rowCacheService.getRowLevelBarrier(key)); // Evict the row cache - region.getBlockCache().evictBlock(key); - assertNull(blockCache.getBlock(key, true, false, true)); + rowCache.evictBlock(key); + assertNull(rowCache.getBlock(key, true)); // Create a row level barrier for the row key rowCacheService.createRowLevelBarrier(key); @@ -123,7 +122,7 @@ public void testBarrier() throws IOException { // Verify that no row cache populated after creating a row level barrier rowCacheService.getScanner(region, get, scan, results); - assertNull(blockCache.getBlock(key, true, false, true)); + assertNull(rowCache.getBlock(key, true)); // Remove the row level barrier rowCacheService.removeRowLevelBarrier(key); @@ -131,12 +130,12 @@ public void testBarrier() throws IOException { // Verify that row cache populated before creating a table level barrier rowCacheService.getScanner(region, get, scan, results); - assertNotNull(blockCache.getBlock(key, true, false, true)); + assertNotNull(rowCache.getBlock(key, true)); assertNull(rowCacheService.getTableLevelBarrier(tableName)); // Evict the row cache - region.getBlockCache().evictBlock(key); - assertNull(blockCache.getBlock(key, true, false, true)); + rowCache.evictBlock(key); + assertNull(rowCache.getBlock(key, true)); // Create a table level barrier for the row key rowCacheService.createTableLevelBarrier(tableName); @@ -144,7 +143,7 @@ public void testBarrier() throws IOException { // Verify that no row cache populated after creating a table level barrier rowCacheService.getScanner(region, get, scan, results); - assertNull(blockCache.getBlock(key, true, false, true)); + assertNull(rowCache.getBlock(key, true)); // Remove the table level barrier rowCacheService.removeTableLevelBarrier(tableName); @@ -204,7 +203,7 @@ public void testMutate() throws IOException, ServiceException { activePolicyEnforcement, rpcCallContext); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -215,7 +214,7 @@ public void testMutate() throws IOException, ServiceException { rowCacheService.checkAndMutate(region, checkAndMutate, 0, 0); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -227,7 +226,7 @@ public void testMutate() throws IOException, ServiceException { rowCacheService.checkAndMutate(region, mutations, checkAndMutate, 0, 0); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -239,7 +238,7 @@ public void testMutate() throws IOException, ServiceException { rowCacheService.batchMutate(region, mutationArray, true, 0, 0); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any(), Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -260,4 +259,59 @@ public void testMutate() throws IOException, ServiceException { inOrder.verify(rowCacheService, Mockito.times(1)).bulkLoad(Mockito.any(), Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeTableLevelBarrier(Mockito.any()); } + + @Test + public void testCaching() throws IOException { + // Mocking dependencies to create RowCacheService instance + RegionInfo regionInfo = Mockito.mock(RegionInfo.class); + Mockito.when(regionInfo.getEncodedName()).thenReturn("region1"); + TableName tableName = TableName.valueOf("table1"); + Mockito.when(regionInfo.getTable()).thenReturn(tableName); + + List stores = new ArrayList<>(); + HStore hStore = Mockito.mock(HStore.class); + Mockito.when(hStore.getStorefilesCount()).thenReturn(2); + stores.add(hStore); + + ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder("CF1".getBytes()).build(); + TableDescriptor td = Mockito.mock(TableDescriptor.class); + Mockito.when(td.isRowCacheEnabled()).thenReturn(true); + Mockito.when(td.getColumnFamilies()).thenReturn(new ColumnFamilyDescriptor[] { cfd }); + + byte[] rowKey = "row".getBytes(); + RegionScannerImpl regionScanner = Mockito.mock(RegionScannerImpl.class); + + Get get = new Get(rowKey); + Scan scan = new Scan(get); + + Configuration conf = HBaseConfiguration.create(); + conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); + HRegion region = Mockito.mock(HRegion.class); + Mockito.when(region.getRegionInfo()).thenReturn(regionInfo); + Mockito.when(region.getTableDescriptor()).thenReturn(td); + Mockito.when(region.getStores()).thenReturn(stores); + Mockito.when(region.getScanner(scan)).thenReturn(regionScanner); + Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + + RowCacheKey key = new RowCacheKey(region, rowKey); + List results = new ArrayList<>(); + results.add(KeyValueTestUtil.create("row", "CF", "q1", 1, "v1")); + + // Initialize RowCacheService + RowCacheService rowCacheService = new RowCacheService(conf); + RowCache rowCache = rowCacheService.getRowCache(); + + // Verify that row cache populated with caching=false + // This should be called first not to populate the row cache + get.setCacheBlocks(false); + rowCacheService.getScanner(region, get, scan, results); + assertNull(rowCache.getBlock(key, true)); + assertNull(rowCache.getBlock(key, false)); + + // Verify that row cache populated with caching=true + get.setCacheBlocks(true); + rowCacheService.getScanner(region, get, scan, results); + assertNotNull(rowCache.getBlock(key, true)); + assertNull(rowCache.getBlock(key, false)); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java index 61ed58f28110..50bed1ab66b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java @@ -17,7 +17,9 @@ */ package org.apache.hadoop.hbase.tool; +import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -36,8 +38,9 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.io.hfile.RowCacheKey; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.RowCache; +import org.apache.hadoop.hbase.regionserver.RowCacheKey; import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -104,8 +107,14 @@ private Path buildBulkFiles(TableName table) throws Exception { @BeforeClass public static void setupCluster() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + + // Enable row cache but reduce the block cache size to fit in 80% of the heap + conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); + conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); + // To test simply, regardless of the number of HFiles - TEST_UTIL.getConfiguration().setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); + conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); TEST_UTIL.startMiniCluster(1); admin = TEST_UTIL.getAdmin(); } @@ -134,6 +143,9 @@ public void after() throws Exception { @Test public void testRowCache() throws Exception { + RowCache rowCache = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRSRpcServices() + .getRowCacheService().getRowCache(); + // Put a row to populate a row cache Put put = new Put("row".getBytes()); put.addColumn(family(0).getBytes(), "q1".getBytes(), "value".getBytes()); @@ -143,7 +155,7 @@ public void testRowCache() throws Exception { Get get = new Get("row".getBytes()); table.get(get); RowCacheKey keyPrev = new RowCacheKey(region, get.getRow()); - assertNotNull(region.getBlockCache().getBlock(keyPrev, true, false, true)); + assertNotNull(rowCache.getBlock(keyPrev, true)); // bulkload Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); @@ -153,8 +165,8 @@ public void testRowCache() throws Exception { // Ensure the row cache is removed after bulkload RowCacheKey keyCur = new RowCacheKey(region, get.getRow()); - assertNull(region.getBlockCache().getBlock(keyCur, true, false, true)); + assertNull(rowCache.getBlock(keyCur, true)); // Ensure the row cache for keyPrev still exists, but it is not used any more. - assertNotNull(region.getBlockCache().getBlock(keyPrev, true, false, true)); + assertNotNull(rowCache.getBlock(keyPrev, true)); } } From 7429c3b8d6c37259078301709bd512b712ac654d Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Thu, 25 Sep 2025 09:44:28 +0900 Subject: [PATCH 04/21] Invalidate only the row cache of regions that were bulkloaded --- .../hbase/regionserver/RowCacheService.java | 29 ++++---- .../regionserver/TestRowCacheService.java | 12 +-- .../tool/TestBulkLoadHFilesRowCache.java | 73 +++++++++++++------ 3 files changed, 71 insertions(+), 43 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index a485e19538c8..319fb3ad76a0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.CheckAndMutate; import org.apache.hadoop.hbase.client.CheckAndMutateResult; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -56,11 +55,11 @@ @org.apache.yetus.audience.InterfaceAudience.Private public class RowCacheService { /** - * A barrier that prevents the row cache from being populated during table operations, such as - * bulk loads. It is implemented as a counter to address issues that arise when the same table is + * A barrier that prevents the row cache from being populated during region operations, such as + * bulk loads. It is implemented as a counter to address issues that arise when the same region is * updated concurrently. */ - private final Map tableLevelBarrierMap = new ConcurrentHashMap<>(); + private final Map regionLevelBarrierMap = new ConcurrentHashMap<>(); /** * A barrier that prevents the row cache from being populated during row mutations. It is * implemented as a counter to address issues that arise when the same row is mutated @@ -144,8 +143,8 @@ private boolean hasSufficientHFiles(HRegion region) { } private void populateCache(HRegion region, List results, RowCacheKey key) { - // The row cache is populated only when no table level barriers remain - tableLevelBarrierMap.computeIfAbsent(region.getRegionInfo().getTable(), t -> { + // The row cache is populated only when no region level barriers remain + regionLevelBarrierMap.computeIfAbsent(region, t -> { // The row cache is populated only when no row level barriers remain rowLevelBarrierMap.computeIfAbsent(key, k -> { try { @@ -174,7 +173,7 @@ BulkLoadHFileResponse bulkLoadHFile(RSRpcServices rsRpcServices, BulkLoadHFileRe // Since bulkload modifies the store files, the row cache should be disabled until the bulkload // is finished. - createTableLevelBarrier(region.getRegionInfo().getTable()); + createRegionLevelBarrier(region); try { // We do not invalidate the entire row cache directly, as it contains a large number of // entries and takes a long time. Instead, we increment rowCacheSeqNum, which is used when @@ -182,8 +181,8 @@ BulkLoadHFileResponse bulkLoadHFile(RSRpcServices rsRpcServices, BulkLoadHFileRe increaseRowCacheSeqNum(region); return bulkLoad(rsRpcServices, request); } finally { - // The row cache for the table has been enabled again - removeTableLevelBarrier(region.getRegionInfo().getTable()); + // The row cache for the region has been enabled again + removeTableLevelBarrier(region); } } @@ -196,15 +195,15 @@ void increaseRowCacheSeqNum(HRegion region) { region.increaseRowCacheSeqNum(); } - void removeTableLevelBarrier(TableName tableName) { - tableLevelBarrierMap.computeIfPresent(tableName, (k, counter) -> { + void removeTableLevelBarrier(HRegion region) { + regionLevelBarrierMap.computeIfPresent(region, (k, counter) -> { int remaining = counter.decrementAndGet(); return (remaining <= 0) ? null : counter; }); } - void createTableLevelBarrier(TableName tableName) { - tableLevelBarrierMap.computeIfAbsent(tableName, k -> new AtomicInteger(0)).incrementAndGet(); + void createRegionLevelBarrier(HRegion region) { + regionLevelBarrierMap.computeIfAbsent(region, k -> new AtomicInteger(0)).incrementAndGet(); } // @formatter:off @@ -361,8 +360,8 @@ AtomicInteger getRowLevelBarrier(RowCacheKey key) { } // For testing only - AtomicInteger getTableLevelBarrier(TableName tableName) { - return tableLevelBarrierMap.get(tableName); + AtomicInteger getRegionLevelBarrier(HRegion region) { + return regionLevelBarrierMap.get(region); } // For testing only diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java index f396f8bc248b..f226b37d0c95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java @@ -131,23 +131,23 @@ public void testBarrier() throws IOException { // Verify that row cache populated before creating a table level barrier rowCacheService.getScanner(region, get, scan, results); assertNotNull(rowCache.getBlock(key, true)); - assertNull(rowCacheService.getTableLevelBarrier(tableName)); + assertNull(rowCacheService.getRegionLevelBarrier(region)); // Evict the row cache rowCache.evictBlock(key); assertNull(rowCache.getBlock(key, true)); // Create a table level barrier for the row key - rowCacheService.createTableLevelBarrier(tableName); - assertEquals(1, rowCacheService.getTableLevelBarrier(tableName).get()); + rowCacheService.createRegionLevelBarrier(region); + assertEquals(1, rowCacheService.getRegionLevelBarrier(region).get()); // Verify that no row cache populated after creating a table level barrier rowCacheService.getScanner(region, get, scan, results); assertNull(rowCache.getBlock(key, true)); // Remove the table level barrier - rowCacheService.removeTableLevelBarrier(tableName); - assertNull(rowCacheService.getTableLevelBarrier(tableName)); + rowCacheService.removeTableLevelBarrier(region); + assertNull(rowCacheService.getRegionLevelBarrier(region)); } @Test @@ -254,7 +254,7 @@ public void testMutate() throws IOException, ServiceException { inOrder = Mockito.inOrder(rowCacheService); rowCacheService.bulkLoadHFile(rsRpcServices, bulkLoadRequest); // Verify the sequence of method calls - inOrder.verify(rowCacheService, Mockito.times(1)).createTableLevelBarrier(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).createRegionLevelBarrier(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).increaseRowCacheSeqNum(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).bulkLoad(Mockito.any(), Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeTableLevelBarrier(Mockito.any()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java index 50bed1ab66b2..80750f3543c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java @@ -20,10 +20,13 @@ import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import java.io.IOException; +import java.util.Comparator; import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -35,6 +38,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; @@ -61,7 +65,7 @@ public class TestBulkLoadHFilesRowCache { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestBulkLoadHFilesRowCache.class); - private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();; + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private static Admin admin; final static int NUM_CFS = 2; @@ -70,7 +74,7 @@ public class TestBulkLoadHFilesRowCache { private TableName tableName; private Table table; - private HRegion region; + private HRegion[] regions; @Rule public TestName testName = new TestName(); @@ -127,10 +131,15 @@ public static void teardownCluster() throws Exception { @Before public void before() throws Exception { tableName = TableName.valueOf(testName.getMethodName()); - admin.createTable(createTableDesc(tableName)); + // Split the table into 2 regions + byte[][] splitKeys = new byte[][] { TestHRegionServerBulkLoad.rowkey(ROWCOUNT) }; + admin.createTable(createTableDesc(tableName), splitKeys); table = TEST_UTIL.getConnection().getTable(tableName); - region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() - .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); + // Sorted by region name + regions = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)) + .sorted(Comparator.comparing(r -> r.getRegionInfo().getRegionNameAsString())) + .toArray(HRegion[]::new); } @After @@ -146,27 +155,47 @@ public void testRowCache() throws Exception { RowCache rowCache = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRSRpcServices() .getRowCacheService().getRowCache(); - // Put a row to populate a row cache - Put put = new Put("row".getBytes()); - put.addColumn(family(0).getBytes(), "q1".getBytes(), "value".getBytes()); - table.put(put); - - // Ensure the region has a row cache - Get get = new Get("row".getBytes()); - table.get(get); - RowCacheKey keyPrev = new RowCacheKey(region, get.getRow()); - assertNotNull(rowCache.getBlock(keyPrev, true)); - - // bulkload + // The region to be bulk-loaded + byte[] rowKeyRegion0 = TestHRegionServerBulkLoad.rowkey(0); + // The region not to be bulk-loaded + byte[] rowKeyRegion1 = TestHRegionServerBulkLoad.rowkey(ROWCOUNT); + + // Put a row into each region to populate the row cache + Put put0 = new Put(rowKeyRegion0); + put0.addColumn(family(0).getBytes(), "q1".getBytes(), "value".getBytes()); + table.put(put0); + Put put1 = new Put(rowKeyRegion1); + put1.addColumn(family(0).getBytes(), "q1".getBytes(), "value".getBytes()); + table.put(put1); + + // Ensure each region has a row cache + Get get0 = new Get(rowKeyRegion0); + Result result0 = table.get(get0); + assertNotNull(result0); + RowCacheKey keyPrev0 = new RowCacheKey(regions[0], get0.getRow()); + assertNotNull(rowCache.getBlock(keyPrev0, true)); + Get get1 = new Get(rowKeyRegion1); + Result result1 = table.get(get1); + assertNotNull(result1); + RowCacheKey keyPrev1 = new RowCacheKey(regions[1], get1.getRow()); + assertNotNull(rowCache.getBlock(keyPrev1, true)); + + // Do bulkload to region0 only Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf); Path dir = buildBulkFiles(tableName); loader.bulkLoad(tableName, dir); - // Ensure the row cache is removed after bulkload - RowCacheKey keyCur = new RowCacheKey(region, get.getRow()); - assertNull(rowCache.getBlock(keyCur, true)); - // Ensure the row cache for keyPrev still exists, but it is not used any more. - assertNotNull(rowCache.getBlock(keyPrev, true)); + // Ensure the row cache is removed after bulkload for region0 + RowCacheKey keyCur0 = new RowCacheKey(regions[0], get0.getRow()); + assertNotEquals(keyPrev0, keyCur0); + assertNull(rowCache.getBlock(keyCur0, true)); + // Ensure the row cache for keyPrev0 still exists, but it is not used anymore. + assertNotNull(rowCache.getBlock(keyPrev0, true)); + + // Ensure the row cache for region1 is not affected + RowCacheKey keyCur1 = new RowCacheKey(regions[1], get1.getRow()); + assertEquals(keyPrev1, keyCur1); + assertNotNull(rowCache.getBlock(keyCur1, true)); } } From 6df768af9e718e964b3885ca846a16e77117c805 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Thu, 25 Sep 2025 10:07:43 +0900 Subject: [PATCH 05/21] Minor fix --- .../MetricsRegionServerWrapperImpl.java | 2 +- .../hbase/regionserver/RowCacheService.java | 1 - .../hbase/regionserver/TestRowCache.java | 21 +++++++------------ .../regionserver/TestRowCacheCanCacheRow.java | 4 +++- 4 files changed, 11 insertions(+), 17 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index 45651624e63c..d5dd2ca74de6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -68,7 +68,7 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { private BlockCache l1Cache = null; private BlockCache l2Cache = null; private MobFileCache mobFileCache; - private RowCache rowCache; + private final RowCache rowCache; private CacheStats cacheStats; private CacheStats l1Stats = null; private CacheStats l2Stats = null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index 319fb3ad76a0..f011c5bf000f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -364,7 +364,6 @@ AtomicInteger getRegionLevelBarrier(HRegion region) { return regionLevelBarrierMap.get(region); } - // For testing only public RowCache getRowCache() { return rowCache; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java index 16443c4a95ea..5cda7ecb6ee9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java @@ -93,6 +93,7 @@ public class TestRowCache { private TableName tableName; private Table table; + HRegion region; private final Map counterBase = new HashMap<>(); @Rule @@ -138,6 +139,8 @@ public void beforeTestMethod() throws Exception { .setColumnFamily(cf1).setColumnFamily(cf2).build(); admin.createTable(td); table = admin.getConnection().getTable(tableName); + region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() + .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); } @After @@ -168,13 +171,11 @@ private static void recomputeMetrics() { } @Test - public void testGetWithRowCache() throws IOException, InterruptedException { + public void testGetWithRowCache() throws IOException { byte[] rowKey = "row".getBytes(); Get get = new Get(rowKey); Result result; - HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() - .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); RowCacheKey rowCacheKey = new RowCacheKey(region, rowKey); // Put a row @@ -288,15 +289,13 @@ public void testPutWithTTL() throws IOException { } @Test - public void testCheckAndMutate() throws IOException, InterruptedException { + public void testCheckAndMutate() throws IOException { byte[] rowKey = "row".getBytes(); Get get = new Get(rowKey); Result result; CheckAndMutate cam; CheckAndMutateResult camResult; - HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() - .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); RowCacheKey rowCacheKey = new RowCacheKey(region, rowKey); // Put a row @@ -333,7 +332,7 @@ public void testCheckAndMutate() throws IOException, InterruptedException { } @Test - public void testCheckAndMutates() throws IOException, InterruptedException { + public void testCheckAndMutates() throws IOException { byte[] rowKey1 = "row1".getBytes(); byte[] rowKey2 = "row2".getBytes(); Get get1 = new Get(rowKey1); @@ -342,8 +341,6 @@ public void testCheckAndMutates() throws IOException, InterruptedException { List cams; List camResults; - HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() - .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); RowCacheKey rowCacheKey1 = new RowCacheKey(region, rowKey1); RowCacheKey rowCacheKey2 = new RowCacheKey(region, rowKey2); @@ -379,15 +376,13 @@ public void testCheckAndMutates() throws IOException, InterruptedException { } @Test - public void testRowMutations() throws IOException, InterruptedException { + public void testRowMutations() throws IOException { byte[] rowKey1 = "row1".getBytes(); byte[] rowKey2 = "row2".getBytes(); Get get1 = new Get(rowKey1); Get get2 = new Get(rowKey2); Result result1, result2; - HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() - .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); RowCacheKey rowCacheKey1 = new RowCacheKey(region, rowKey1); RowCacheKey rowCacheKey2 = new RowCacheKey(region, rowKey2); @@ -447,8 +442,6 @@ public void testBatch() throws IOException, InterruptedException { List batchOperations; Object[] results; - HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions().stream() - .filter(r -> r.getRegionInfo().getTable().equals(tableName)).findFirst().orElseThrow(); RowCacheKey rowCacheKey1 = new RowCacheKey(region, rowKey1); RowCacheKey rowCacheKey2 = new RowCacheKey(region, rowKey2); RowCacheKey rowCacheKey3 = new RowCacheKey(region, rowKey3); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java index 5c51b63c304b..af1e8c5c3405 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java @@ -252,8 +252,10 @@ private static void testWith(Function func) { get.addFamily(CF1); Assert.assertTrue(rowCacheService.canCacheRow(get, region)); + // noinspection unused + var unused = func.apply(get); + // expect false - Get unused = func.apply(get); Assert.assertFalse(rowCacheService.canCacheRow(get, region)); } } From e33db29e8f7834a26040123afd3e22d6f7c6afca Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Thu, 25 Sep 2025 11:56:37 +0900 Subject: [PATCH 06/21] Do not use the row cache when the data exists only in the MemStore And remove the condition that decides whether to put data into the row cache based on the number of StoreFiles --- .../org/apache/hadoop/hbase/HConstants.java | 8 - .../hbase/regionserver/RSRpcServices.java | 3 +- .../hbase/regionserver/RowCacheService.java | 24 +- .../hbase/regionserver/TestRowCache.java | 47 +++- .../TestRowCacheConfigurationObserver.java | 212 ------------------ .../regionserver/TestRowCacheService.java | 18 +- .../tool/TestBulkLoadHFilesRowCache.java | 4 +- 7 files changed, 63 insertions(+), 253 deletions(-) delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 997342bb6a8d..5381a4e3a18b 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1023,14 +1023,6 @@ public enum OperationStatusCode { public static final String ROW_CACHE_SIZE_KEY = "row.cache.size"; public static final float ROW_CACHE_SIZE_DEFAULT = 0.0f; - /** - * Configuration key for the minimum number of HFiles required to activate the Row Cache. If the - * number of HFiles is less than this value, the Row Cache does not operate even if it is enabled - * at the table level. - */ - public static final String ROW_CACHE_ACTIVATE_MIN_HFILES_KEY = "row.cache.activate.min.hfiles"; - public static final int ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT = 2; - /** * Configuration key for the memory size of the block cache */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index bc777c67df15..f170ed7d9062 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -2583,7 +2583,7 @@ private Result get(Get get, HRegion region, RegionScannersCloseCallBack closeCal RegionScannerImpl scanner = null; long blockBytesScannedBefore = context.getBlockBytesScanned(); try { - scanner = rowCacheService.getScanner(region, get, scan, results); + scanner = rowCacheService.getScanner(region, get, scan, results, context); } finally { if (scanner != null) { if (closeCallBack == null) { @@ -4042,7 +4042,6 @@ private void setReloadableGuardrails(Configuration conf) { public void onConfigurationChange(Configuration conf) { super.onConfigurationChange(conf); setReloadableGuardrails(conf); - rowCacheService.updateConf(conf); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index f011c5bf000f..df75a3b71cbf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -70,28 +70,19 @@ public class RowCacheService { private final boolean enabledByConf; private final RowCache rowCache; - private int activateMinHFiles; - @FunctionalInterface interface RowOperation { R execute() throws IOException; } RowCacheService(Configuration conf) { - updateConf(conf); - enabledByConf = conf.getFloat(HConstants.ROW_CACHE_SIZE_KEY, HConstants.ROW_CACHE_SIZE_DEFAULT) > 0; rowCache = new RowCache(MemorySizeUtil.getRowCacheSize(conf)); } - synchronized void updateConf(Configuration conf) { - this.activateMinHFiles = conf.getInt(HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, - HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT); - } - - RegionScannerImpl getScanner(HRegion region, Get get, Scan scan, List results) - throws IOException { + RegionScannerImpl getScanner(HRegion region, Get get, Scan scan, List results, + RpcCallContext context) throws IOException { if (!canCacheRow(get, region)) { return getScannerInternal(region, scan, results); } @@ -106,9 +97,9 @@ RegionScannerImpl getScanner(HRegion region, Get get, Scan scan, List resu RegionScannerImpl scanner = getScannerInternal(region, scan, results); - // The row cache is ineffective when the number of store files is small. If the number - // of store files falls below the minimum threshold, rows will not be cached - if (hasSufficientHFiles(region)) { + // When results came from memstore only, do not populate the row cache + boolean readFromMemStoreOnly = context.getBlockBytesScanned() < 1; + if (!readFromMemStoreOnly) { populateCache(region, results, key); } @@ -137,11 +128,6 @@ private boolean tryGetFromCache(HRegion region, RowCacheKey key, Get get, List store.getStorefilesCount() >= activateMinHFiles); - } - private void populateCache(HRegion region, List results, RowCacheKey key) { // The row cache is populated only when no region level barriers remain regionLevelBarrierMap.computeIfAbsent(region, t -> { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java index 5cda7ecb6ee9..d7cf93d63be2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; -import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.ROW_CACHE_EVICTED_ROW_COUNT; import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.ROW_CACHE_HIT_COUNT; @@ -107,8 +106,6 @@ public static void beforeClass() throws Exception { conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); - // To test simply, regardless of the number of HFiles - conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); cluster.waitForActiveAndReadyMaster(); admin = TEST_UTIL.getAdmin(); @@ -185,6 +182,7 @@ public void testGetWithRowCache() throws IOException { put.addColumn(CF2, Q1, "21".getBytes()); put.addColumn(CF2, Q2, "22".getBytes()); table.put(put); + admin.flush(tableName); // Initialize metrics recomputeMetrics(); @@ -303,6 +301,7 @@ public void testCheckAndMutate() throws IOException { put1.addColumn(CF1, Q1, "11".getBytes()); put1.addColumn(CF1, Q2, "12".getBytes()); table.put(put1); + admin.flush(tableName); // Validate that the row cache is populated result = table.get(get); @@ -353,6 +352,7 @@ public void testCheckAndMutates() throws IOException { put2.addColumn(CF1, Q1, "211".getBytes()); put2.addColumn(CF1, Q2, "212".getBytes()); table.put(put2); + admin.flush(tableName); // Validate that the row caches are populated result1 = table.get(get1); @@ -395,6 +395,7 @@ public void testRowMutations() throws IOException { put2.addColumn(CF1, Q1, "211".getBytes()); put2.addColumn(CF1, Q2, "212".getBytes()); table.put(put2); + admin.flush(tableName); // Validate that the row caches are populated result1 = table.get(get1); @@ -462,6 +463,7 @@ public void testBatch() throws IOException, InterruptedException { batchOperations.add(put3); results = new Result[batchOperations.size()]; table.batch(batchOperations, results); + admin.flush(tableName); // Validate that the row caches are populated batchOperations = new ArrayList<>(); @@ -497,4 +499,43 @@ public void testBatch() throws IOException, InterruptedException { assertNull(rowCache.getBlock(rowCacheKey2, true)); assertNotNull(rowCache.getBlock(rowCacheKey3, true)); } + + @Test + public void testGetFromMemstoreOnly() throws IOException, InterruptedException { + byte[] rowKey = "row".getBytes(); + RowCacheKey rowCacheKey = new RowCacheKey(region, rowKey); + + // Put a row into memstore only, not flushed to HFile yet + Put put = new Put(rowKey); + put.addColumn(CF1, Q1, Bytes.toBytes(0L)); + table.put(put); + + // Get from memstore only + Get get = new Get(rowKey); + table.get(get); + + // Validate that the row cache is not populated + assertNull(rowCache.getBlock(rowCacheKey, true)); + + // Flush memstore to HFile, then get again + admin.flush(tableName); + get = new Get(rowKey); + table.get(get); + + // Validate that the row cache is populated now + assertNotNull(rowCache.getBlock(rowCacheKey, true)); + + // Put another qualifier. And now the cells are in both memstore and HFile. + put = new Put(rowKey); + put.addColumn(CF1, Q2, Bytes.toBytes(0L)); + table.put(put); + + // Validate that the row cache is invalidated + assertNull(rowCache.getBlock(rowCacheKey, true)); + + // Get from memstore and HFile + get = new Get(rowKey); + table.get(get); + assertNotNull(rowCache.getBlock(rowCacheKey, true)); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java deleted file mode 100644 index d0dd09f855b8..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfigurationObserver.java +++ /dev/null @@ -1,212 +0,0 @@ -/* - * 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.regionserver; - -import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; -import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; -import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; -import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.ROW_CACHE_HIT_COUNT; -import static org.junit.Assert.assertArrayEquals; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CompatibilityFactory; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.SingleProcessHBaseCluster; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.test.MetricsAssertHelper; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; - -@Category({ RegionServerTests.class, MediumTests.class }) -public class TestRowCacheConfigurationObserver { - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestRowCacheConfigurationObserver.class); - - private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); - private static final byte[] CF1 = Bytes.toBytes("cf1"); - private static final byte[] Q1 = Bytes.toBytes("q1"); - - private static MetricsAssertHelper metricsHelper; - private static MetricsRegionServer metricsRegionServer; - private static MetricsRegionServerSource serverSource; - - private static Admin admin; - - private TableName tableName; - private Table table; - private final Map counterBase = new HashMap<>(); - - @Rule - public TestName testName = new TestName(); - - @BeforeClass - public static void beforeClass() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - - // Enable row cache but reduce the block cache size to fit in 80% of the heap - conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); - conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); - - // Set a value different from the default - conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); - SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); - cluster.waitForActiveAndReadyMaster(); - admin = TEST_UTIL.getAdmin(); - - metricsHelper = CompatibilityFactory.getInstance(MetricsAssertHelper.class); - metricsRegionServer = cluster.getRegionServer(0).getMetrics(); - serverSource = metricsRegionServer.getMetricsSource(); - } - - @AfterClass - public static void afterClass() throws Exception { - HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false; - TEST_UTIL.shutdownMiniCluster(); - } - - @Before - public void beforeTestMethod() throws Exception { - ColumnFamilyDescriptor cf1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); - - tableName = TableName.valueOf(testName.getMethodName()); - TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName).setRowCacheEnabled(true) - .setColumnFamily(cf1).build(); - admin.createTable(td); - table = admin.getConnection().getTable(tableName); - } - - @After - public void afterTestMethod() throws Exception { - counterBase.clear(); - - admin.disableTable(tableName); - admin.deleteTable(tableName); - } - - private void setCounterBase(String metric, long value) { - counterBase.put(metric, value); - } - - private void assertCounterDiff(String metric, long diff) { - Long base = counterBase.get(metric); - if (base == null) { - throw new IllegalStateException( - "base counter of " + metric + " metric should have been set before by setCounterBase()"); - } - long newValue = base + diff; - metricsHelper.assertCounter(metric, newValue, serverSource); - counterBase.put(metric, newValue); - } - - private static void recomputeMetrics() { - metricsRegionServer.getRegionServerWrapper().forceRecompute(); - } - - /** - * If the number of HFiles is below the configured minimum, row caching has no effect. - */ - @Test - public void testRowCacheWithHFilesCount() throws IOException { - // Change ROW_CACHE_ACTIVATE_MIN_HFILES_KEY online - Configuration conf = TEST_UTIL.getConfiguration(); - conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 2); - for (ServerName serverName : admin.getRegionServers()) { - HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(serverName); - regionServer.getConfigurationManager().notifyAllObservers(conf); - } - - byte[] rowKey = "row".getBytes(); - Get get = new Get(rowKey); - Result result; - - // Initialize metrics - recomputeMetrics(); - setCounterBase("Get_num_ops", metricsHelper.getCounter("Get_num_ops", serverSource)); - setCounterBase(ROW_CACHE_HIT_COUNT, - metricsHelper.getCounter(ROW_CACHE_HIT_COUNT, serverSource)); - - Put put = new Put(rowKey); - put.addColumn(CF1, Q1, "11".getBytes()); - table.put(put); - - // The row cache is not populated yet, as the store file count is 0 - result = table.get(get); - recomputeMetrics(); - assertArrayEquals(rowKey, result.getRow()); - assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); - assertCounterDiff("Get_num_ops", 1); - assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); - - // Flush, 1 store file exists - TEST_UTIL.getAdmin().flush(tableName); - - // The row cache is not populated yet, as the store file count is 1 - result = table.get(get); - recomputeMetrics(); - assertArrayEquals(rowKey, result.getRow()); - assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); - assertCounterDiff("Get_num_ops", 1); - assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); - - // Flush, 2(ROW_CACHE_ACTIVATE_MIN_HFILES_DEFAULT) store files exist - table.put(put); - TEST_UTIL.getAdmin().flush(tableName); - - // The row cache is populated now, as the store file count is 2. - // But the row cache is not hit yet, it will be hit only after this operation. - result = table.get(get); - recomputeMetrics(); - assertArrayEquals(rowKey, result.getRow()); - assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); - assertCounterDiff("Get_num_ops", 1); - assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); - - // Get again, this time the row cache is hit - result = table.get(get); - recomputeMetrics(); - assertArrayEquals(rowKey, result.getRow()); - assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); - assertCounterDiff("Get_num_ops", 1); - assertCounterDiff(ROW_CACHE_HIT_COUNT, 1); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java index f226b37d0c95..57e5c9597189 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java @@ -99,6 +99,9 @@ public void testBarrier() throws IOException { Mockito.when(region.getScanner(scan)).thenReturn(regionScanner); Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + RpcCallContext context = Mockito.mock(RpcCallContext.class); + Mockito.when(context.getBlockBytesScanned()).thenReturn(1L); + RowCacheKey key = new RowCacheKey(region, rowKey); List results = new ArrayList<>(); results.add(KeyValueTestUtil.create("row", "CF", "q1", 1, "v1")); @@ -108,7 +111,7 @@ public void testBarrier() throws IOException { RowCache rowCache = rowCacheService.getRowCache(); // Verify that row cache populated before creating a row level barrier - rowCacheService.getScanner(region, get, scan, results); + rowCacheService.getScanner(region, get, scan, results, context); assertNotNull(rowCache.getBlock(key, true)); assertNull(rowCacheService.getRowLevelBarrier(key)); @@ -121,7 +124,7 @@ public void testBarrier() throws IOException { assertEquals(1, rowCacheService.getRowLevelBarrier(key).get()); // Verify that no row cache populated after creating a row level barrier - rowCacheService.getScanner(region, get, scan, results); + rowCacheService.getScanner(region, get, scan, results, context); assertNull(rowCache.getBlock(key, true)); // Remove the row level barrier @@ -129,7 +132,7 @@ public void testBarrier() throws IOException { assertNull(rowCacheService.getRowLevelBarrier(key)); // Verify that row cache populated before creating a table level barrier - rowCacheService.getScanner(region, get, scan, results); + rowCacheService.getScanner(region, get, scan, results, context); assertNotNull(rowCache.getBlock(key, true)); assertNull(rowCacheService.getRegionLevelBarrier(region)); @@ -142,7 +145,7 @@ public void testBarrier() throws IOException { assertEquals(1, rowCacheService.getRegionLevelBarrier(region).get()); // Verify that no row cache populated after creating a table level barrier - rowCacheService.getScanner(region, get, scan, results); + rowCacheService.getScanner(region, get, scan, results, context); assertNull(rowCache.getBlock(key, true)); // Remove the table level barrier @@ -278,6 +281,9 @@ public void testCaching() throws IOException { Mockito.when(td.isRowCacheEnabled()).thenReturn(true); Mockito.when(td.getColumnFamilies()).thenReturn(new ColumnFamilyDescriptor[] { cfd }); + RpcCallContext context = Mockito.mock(RpcCallContext.class); + Mockito.when(context.getBlockBytesScanned()).thenReturn(1L); + byte[] rowKey = "row".getBytes(); RegionScannerImpl regionScanner = Mockito.mock(RegionScannerImpl.class); @@ -304,13 +310,13 @@ public void testCaching() throws IOException { // Verify that row cache populated with caching=false // This should be called first not to populate the row cache get.setCacheBlocks(false); - rowCacheService.getScanner(region, get, scan, results); + rowCacheService.getScanner(region, get, scan, results, context); assertNull(rowCache.getBlock(key, true)); assertNull(rowCache.getBlock(key, false)); // Verify that row cache populated with caching=true get.setCacheBlocks(true); - rowCacheService.getScanner(region, get, scan, results); + rowCacheService.getScanner(region, get, scan, results, context); assertNotNull(rowCache.getBlock(key, true)); assertNull(rowCache.getBlock(key, false)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java index 80750f3543c6..7c9eb35f28f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.tool; import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; -import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_ACTIVATE_MIN_HFILES_KEY; import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -117,8 +116,6 @@ public static void setupCluster() throws Exception { conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); - // To test simply, regardless of the number of HFiles - conf.setInt(ROW_CACHE_ACTIVATE_MIN_HFILES_KEY, 0); TEST_UTIL.startMiniCluster(1); admin = TEST_UTIL.getAdmin(); } @@ -167,6 +164,7 @@ public void testRowCache() throws Exception { Put put1 = new Put(rowKeyRegion1); put1.addColumn(family(0).getBytes(), "q1".getBytes(), "value".getBytes()); table.put(put1); + admin.flush(tableName); // Ensure each region has a row cache Get get0 = new Get(rowKeyRegion0); From 7091abe2e47a589077adda79d2c36b6005dbfc40 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Sat, 4 Oct 2025 12:18:00 +0900 Subject: [PATCH 07/21] Some cosmetic fix related to RowCache --- .../hbase/tmpl/regionserver/BlockCacheTmpl.jamon | 10 +++++----- .../hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon index b949c1025af3..3aa9ccfc444d 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon @@ -50,9 +50,9 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
@@ -573,7 +573,7 @@ Increase that value to get a complete picture. RowCache rowCache; <%if rowCache == null %> -

RowCache is null

+

RowCache is disabled

<%else> @@ -617,6 +617,6 @@ Increase that value to get a complete picture.
Hit Count divided by total requests count
-

RowCache is a separate cache distinct from BlockCache. Since there is no more appropriate place to display RowCache information, it is shown here alongside BlockCache.

+

RowCache is a separate cache distinct from BlockCache.

diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon index 3351ddee3ef5..906201fb2758 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon @@ -152,7 +152,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-

Block Cache

+

Cache

<& BlockCacheTmpl; cacheConfig = new CacheConfig(regionServer.getConfiguration()); config = regionServer.getConfiguration(); bc = regionServer.getBlockCache().orElse(null); rowCache = rowCache &>
From d54fff4734ffbb41f406003b931c0956a024e68f Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Sat, 4 Oct 2025 12:19:18 +0900 Subject: [PATCH 08/21] Bump Caffeine to the latest version https://github.com/ben-manes/caffeine?tab=readme-ov-file#download It is recommended to use 3.x for Java 11 or above. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 170dae40db9c..0b3c7c4a905d 100644 --- a/pom.xml +++ b/pom.xml @@ -875,7 +875,7 @@ 0.15.0 0.15.0 1.11.4 - 2.8.1 + 3.2.2 1.15 1.7 2.18.0 From 99ce0f0b1ae07d0e4f9dead86d1cf81232b28fc0 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Sun, 5 Oct 2025 11:11:07 +0900 Subject: [PATCH 09/21] Retrigger From e3a8af8dcb2af5f3d91a7a087f1e012ed1b6c991 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Tue, 14 Oct 2025 14:09:58 +0900 Subject: [PATCH 10/21] Enable setting ROW_CACHE_ENABLED via HBase Shell --- hbase-shell/src/main/ruby/hbase/admin.rb | 1 + 1 file changed, 1 insertion(+) diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 5ceaf2a08c72..faabdae11689 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -1604,6 +1604,7 @@ def update_tdb_from_arg(tdb, arg) tdb.setRegionMemStoreReplication(JBoolean.valueOf(arg.delete(TableDescriptorBuilder::REGION_MEMSTORE_REPLICATION))) if arg.include?(TableDescriptorBuilder::REGION_MEMSTORE_REPLICATION) tdb.setRegionSplitPolicyClassName(arg.delete(TableDescriptorBuilder::SPLIT_POLICY)) if arg.include?(TableDescriptorBuilder::SPLIT_POLICY) tdb.setRegionReplication(JInteger.valueOf(arg.delete(TableDescriptorBuilder::REGION_REPLICATION))) if arg.include?(TableDescriptorBuilder::REGION_REPLICATION) + tdb.setRowCacheEnabled(JBoolean.valueOf(arg.delete(TableDescriptorBuilder::ROW_CACHE_ENABLED))) if arg.include?(TableDescriptorBuilder::ROW_CACHE_ENABLED) set_user_metadata(tdb, arg.delete(METADATA)) if arg[METADATA] set_descriptor_config(tdb, arg.delete(CONFIGURATION)) if arg[CONFIGURATION] end From ffba4176f6f2947994ccd2ac94868caaf59cbbf2 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Tue, 14 Oct 2025 14:52:18 +0900 Subject: [PATCH 11/21] Replace manual hit/miss counting with Cache.stats() --- .../hadoop/hbase/regionserver/RowCache.java | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java index d5b1cf3ae002..77b0a46608bb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java @@ -43,8 +43,7 @@ public void onRemoval(RowCacheKey key, RowCells value, @NonNull RemovalCause cau private final Cache<@NonNull RowCacheKey, RowCells> cache; - private final LongAdder hitCount = new LongAdder(); - private final LongAdder missCount = new LongAdder(); + // Cache.stats() does not provide eviction count for entries, so we maintain our own counter. private final LongAdder evictedRowCount = new LongAdder(); RowCache(long maxSizeBytes) { @@ -57,6 +56,7 @@ public void onRemoval(RowCacheKey key, RowCells value, @NonNull RemovalCause cau Caffeine.newBuilder().maximumWeight(maxSizeBytes).removalListener(new EvictionListener()) .weigher((RowCacheKey key, RowCells value) -> (int) Math.min(key.heapSize() + value.heapSize(), Integer.MAX_VALUE)) + .recordStats() .build(); } @@ -66,17 +66,10 @@ void cacheBlock(RowCacheKey key, RowCells value) { public RowCells getBlock(RowCacheKey key, boolean caching) { if (!caching) { - missCount.increment(); return null; } - RowCells value = cache.getIfPresent(key); - if (value == null) { - missCount.increment(); - } else { - hitCount.increment(); - } - return value; + return cache.getIfPresent(key); } void evictBlock(RowCacheKey key) { @@ -84,11 +77,11 @@ void evictBlock(RowCacheKey key) { } public long getHitCount() { - return hitCount.sum(); + return cache.stats().hitCount(); } public long getMissCount() { - return missCount.sum(); + return cache.stats().missCount(); } public long getEvictedRowCount() { From 95d7892f49d35aac5c869a5e956046904c46d040 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Tue, 14 Oct 2025 14:59:29 +0900 Subject: [PATCH 12/21] Rename method for RowCache --- .../hadoop/hbase/regionserver/RowCache.java | 6 +- .../hbase/regionserver/RowCacheService.java | 12 ++-- .../hbase/regionserver/TestRowCache.java | 66 +++++++++---------- .../regionserver/TestRowCacheService.java | 32 ++++----- .../tool/TestBulkLoadHFilesRowCache.java | 10 +-- 5 files changed, 63 insertions(+), 63 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java index 77b0a46608bb..d7914a1788ff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java @@ -60,11 +60,11 @@ public void onRemoval(RowCacheKey key, RowCells value, @NonNull RemovalCause cau .build(); } - void cacheBlock(RowCacheKey key, RowCells value) { + void cacheRow(RowCacheKey key, RowCells value) { cache.put(key, value); } - public RowCells getBlock(RowCacheKey key, boolean caching) { + public RowCells getRow(RowCacheKey key, boolean caching) { if (!caching) { return null; } @@ -72,7 +72,7 @@ public RowCells getBlock(RowCacheKey key, boolean caching) { return cache.getIfPresent(key); } - void evictBlock(RowCacheKey key) { + void evictRow(RowCacheKey key) { cache.asMap().remove(key); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index df75a3b71cbf..655ad4d646bc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -114,7 +114,7 @@ private RegionScannerImpl getScannerInternal(HRegion region, Scan scan, List results) { - RowCells row = rowCache.getBlock(key, get.getCacheBlocks()); + RowCells row = rowCache.getRow(key, get.getCacheBlocks()); if (row == null) { return false; @@ -134,7 +134,7 @@ private void populateCache(HRegion region, List results, RowCacheKey key) // The row cache is populated only when no row level barriers remain rowLevelBarrierMap.computeIfAbsent(key, k -> { try { - rowCache.cacheBlock(key, new RowCells(results)); + rowCache.cacheRow(key, new RowCells(results)); } catch (CloneNotSupportedException ignored) { // Not able to cache row cells, ignore } @@ -254,7 +254,7 @@ private R mutateWithRowCacheBarrier(HRegion region, List mutations // After creating the barrier, evict the existing row cache for this row, // as it becomes invalid after the mutation - evictRowCache(key); + evictRow(key); }); return execute(operation); @@ -278,7 +278,7 @@ private R mutateWithRowCacheBarrier(HRegion region, byte[] row, RowOperation // After creating the barrier, evict the existing row cache for this row, // as it becomes invalid after the mutation - evictRowCache(key); + evictRow(key); return execute(operation); } finally { @@ -291,8 +291,8 @@ R execute(RowOperation operation) throws IOException { return operation.execute(); } - void evictRowCache(RowCacheKey key) { - rowCache.evictBlock(key); + void evictRow(RowCacheKey key) { + rowCache.evictRow(key); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java index d7cf93d63be2..7d1115e7d85a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java @@ -221,7 +221,7 @@ public void testGetWithRowCache() throws IOException { assertCounterDiff(ROW_CACHE_MISS_COUNT, 0); // Row cache is invalidated by the put operation - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); table.put(put); recomputeMetrics(); assertCounterDiff(ROW_CACHE_HIT_COUNT, 1); @@ -249,31 +249,31 @@ public void testGetWithRowCache() throws IOException { assertCounterDiff(ROW_CACHE_EVICTED_ROW_COUNT, 0); // Row cache is invalidated by the increment operation - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); table.incrementColumnValue(rowKey, CF1, Q1, 1); - assertNull(rowCache.getBlock(rowCacheKey, true)); + assertNull(rowCache.getRow(rowCacheKey, true)); // Get is executed without the row cache; however, the cache is re-populated as a result table.get(get); - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); // Row cache is invalidated by the append operation - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); Append append = new Append(rowKey); append.addColumn(CF1, Q1, Bytes.toBytes(0L)); table.append(append); - assertNull(rowCache.getBlock(rowCacheKey, true)); + assertNull(rowCache.getRow(rowCacheKey, true)); // Get is executed without the row cache; however, the cache is re-populated as a result table.get(get); - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); // Row cache is invalidated by the delete operation - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); Delete delete = new Delete(rowKey); delete.addColumn(CF1, Q1); table.delete(delete); - assertNull(rowCache.getBlock(rowCacheKey, true)); + assertNull(rowCache.getRow(rowCacheKey, true)); } @Test(expected = DoNotRetryIOException.class) @@ -305,7 +305,7 @@ public void testCheckAndMutate() throws IOException { // Validate that the row cache is populated result = table.get(get); - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); @@ -315,11 +315,11 @@ public void testCheckAndMutate() throws IOException { cam = CheckAndMutate.newBuilder(rowKey).ifEquals(CF1, Q2, "00".getBytes()).build(put2); camResult = table.checkAndMutate(cam); assertFalse(camResult.isSuccess()); - assertNull(rowCache.getBlock(rowCacheKey, true)); + assertNull(rowCache.getRow(rowCacheKey, true)); // Validate that the row cache is populated result = table.get(get); - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); assertArrayEquals("11".getBytes(), result.getValue(CF1, Q1)); assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); @@ -327,7 +327,7 @@ public void testCheckAndMutate() throws IOException { cam = CheckAndMutate.newBuilder(rowKey).ifEquals(CF1, Q2, "12".getBytes()).build(put2); camResult = table.checkAndMutate(cam); assertTrue(camResult.isSuccess()); - assertNull(rowCache.getBlock(rowCacheKey, true)); + assertNull(rowCache.getRow(rowCacheKey, true)); } @Test @@ -356,11 +356,11 @@ public void testCheckAndMutates() throws IOException { // Validate that the row caches are populated result1 = table.get(get1); - assertNotNull(rowCache.getBlock(rowCacheKey1, true)); + assertNotNull(rowCache.getRow(rowCacheKey1, true)); assertArrayEquals("111".getBytes(), result1.getValue(CF1, Q1)); assertArrayEquals("112".getBytes(), result1.getValue(CF1, Q2)); result2 = table.get(get2); - assertNotNull(rowCache.getBlock(rowCacheKey2, true)); + assertNotNull(rowCache.getRow(rowCacheKey2, true)); assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); @@ -371,8 +371,8 @@ public void testCheckAndMutates() throws IOException { camResults = table.checkAndMutate(cams); assertTrue(camResults.get(0).isSuccess()); assertTrue(camResults.get(1).isSuccess()); - assertNull(rowCache.getBlock(rowCacheKey1, true)); - assertNull(rowCache.getBlock(rowCacheKey2, true)); + assertNull(rowCache.getRow(rowCacheKey1, true)); + assertNull(rowCache.getRow(rowCacheKey2, true)); } @Test @@ -399,11 +399,11 @@ public void testRowMutations() throws IOException { // Validate that the row caches are populated result1 = table.get(get1); - assertNotNull(rowCache.getBlock(rowCacheKey1, true)); + assertNotNull(rowCache.getRow(rowCacheKey1, true)); assertArrayEquals("111".getBytes(), result1.getValue(CF1, Q1)); assertArrayEquals("112".getBytes(), result1.getValue(CF1, Q2)); result2 = table.get(get2); - assertNotNull(rowCache.getBlock(rowCacheKey1, true)); + assertNotNull(rowCache.getRow(rowCacheKey1, true)); assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); @@ -418,16 +418,16 @@ public void testRowMutations() throws IOException { CheckAndMutate cam = CheckAndMutate.newBuilder(rowKey1).ifEquals(CF1, Q1, "111".getBytes()).build(rms); table.checkAndMutate(cam); - assertNull(rowCache.getBlock(rowCacheKey1, true)); - assertNotNull(rowCache.getBlock(rowCacheKey2, true)); + assertNull(rowCache.getRow(rowCacheKey1, true)); + assertNotNull(rowCache.getRow(rowCacheKey2, true)); // Validate that the row caches are populated result1 = table.get(get1); - assertNotNull(rowCache.getBlock(rowCacheKey1, true)); + assertNotNull(rowCache.getRow(rowCacheKey1, true)); assertArrayEquals("111111".getBytes(), result1.getValue(CF1, Q1)); assertArrayEquals("112112".getBytes(), result1.getValue(CF1, Q2)); result2 = table.get(get2); - assertNotNull(rowCache.getBlock(rowCacheKey1, true)); + assertNotNull(rowCache.getRow(rowCacheKey1, true)); assertArrayEquals("211".getBytes(), result2.getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), result2.getValue(CF1, Q2)); } @@ -473,13 +473,13 @@ public void testBatch() throws IOException, InterruptedException { results = new Object[batchOperations.size()]; table.batch(batchOperations, results); assertEquals(3, results.length); - assertNotNull(rowCache.getBlock(rowCacheKey1, true)); + assertNotNull(rowCache.getRow(rowCacheKey1, true)); assertArrayEquals("111".getBytes(), ((Result) results[0]).getValue(CF1, Q1)); assertArrayEquals("112".getBytes(), ((Result) results[0]).getValue(CF1, Q2)); - assertNotNull(rowCache.getBlock(rowCacheKey2, true)); + assertNotNull(rowCache.getRow(rowCacheKey2, true)); assertArrayEquals("211".getBytes(), ((Result) results[1]).getValue(CF1, Q1)); assertArrayEquals("212".getBytes(), ((Result) results[1]).getValue(CF1, Q2)); - assertNotNull(rowCache.getBlock(rowCacheKey3, true)); + assertNotNull(rowCache.getRow(rowCacheKey3, true)); assertArrayEquals("311".getBytes(), ((Result) results[2]).getValue(CF1, Q1)); assertArrayEquals("312".getBytes(), ((Result) results[2]).getValue(CF1, Q2)); @@ -495,9 +495,9 @@ public void testBatch() throws IOException, InterruptedException { results = new Object[batchOperations.size()]; table.batch(batchOperations, results); assertEquals(2, results.length); - assertNull(rowCache.getBlock(rowCacheKey1, true)); - assertNull(rowCache.getBlock(rowCacheKey2, true)); - assertNotNull(rowCache.getBlock(rowCacheKey3, true)); + assertNull(rowCache.getRow(rowCacheKey1, true)); + assertNull(rowCache.getRow(rowCacheKey2, true)); + assertNotNull(rowCache.getRow(rowCacheKey3, true)); } @Test @@ -515,7 +515,7 @@ public void testGetFromMemstoreOnly() throws IOException, InterruptedException { table.get(get); // Validate that the row cache is not populated - assertNull(rowCache.getBlock(rowCacheKey, true)); + assertNull(rowCache.getRow(rowCacheKey, true)); // Flush memstore to HFile, then get again admin.flush(tableName); @@ -523,7 +523,7 @@ public void testGetFromMemstoreOnly() throws IOException, InterruptedException { table.get(get); // Validate that the row cache is populated now - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); // Put another qualifier. And now the cells are in both memstore and HFile. put = new Put(rowKey); @@ -531,11 +531,11 @@ public void testGetFromMemstoreOnly() throws IOException, InterruptedException { table.put(put); // Validate that the row cache is invalidated - assertNull(rowCache.getBlock(rowCacheKey, true)); + assertNull(rowCache.getRow(rowCacheKey, true)); // Get from memstore and HFile get = new Get(rowKey); table.get(get); - assertNotNull(rowCache.getBlock(rowCacheKey, true)); + assertNotNull(rowCache.getRow(rowCacheKey, true)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java index 57e5c9597189..6e1c6c83a0e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java @@ -112,12 +112,12 @@ public void testBarrier() throws IOException { // Verify that row cache populated before creating a row level barrier rowCacheService.getScanner(region, get, scan, results, context); - assertNotNull(rowCache.getBlock(key, true)); + assertNotNull(rowCache.getRow(key, true)); assertNull(rowCacheService.getRowLevelBarrier(key)); // Evict the row cache - rowCache.evictBlock(key); - assertNull(rowCache.getBlock(key, true)); + rowCache.evictRow(key); + assertNull(rowCache.getRow(key, true)); // Create a row level barrier for the row key rowCacheService.createRowLevelBarrier(key); @@ -125,7 +125,7 @@ public void testBarrier() throws IOException { // Verify that no row cache populated after creating a row level barrier rowCacheService.getScanner(region, get, scan, results, context); - assertNull(rowCache.getBlock(key, true)); + assertNull(rowCache.getRow(key, true)); // Remove the row level barrier rowCacheService.removeRowLevelBarrier(key); @@ -133,12 +133,12 @@ public void testBarrier() throws IOException { // Verify that row cache populated before creating a table level barrier rowCacheService.getScanner(region, get, scan, results, context); - assertNotNull(rowCache.getBlock(key, true)); + assertNotNull(rowCache.getRow(key, true)); assertNull(rowCacheService.getRegionLevelBarrier(region)); // Evict the row cache - rowCache.evictBlock(key); - assertNull(rowCache.getBlock(key, true)); + rowCache.evictRow(key); + assertNull(rowCache.getRow(key, true)); // Create a table level barrier for the row key rowCacheService.createRegionLevelBarrier(region); @@ -146,7 +146,7 @@ public void testBarrier() throws IOException { // Verify that no row cache populated after creating a table level barrier rowCacheService.getScanner(region, get, scan, results, context); - assertNull(rowCache.getBlock(key, true)); + assertNull(rowCache.getRow(key, true)); // Remove the table level barrier rowCacheService.removeTableLevelBarrier(region); @@ -206,7 +206,7 @@ public void testMutate() throws IOException, ServiceException { activePolicyEnforcement, rpcCallContext); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRow(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -217,7 +217,7 @@ public void testMutate() throws IOException, ServiceException { rowCacheService.checkAndMutate(region, checkAndMutate, 0, 0); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRow(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -229,7 +229,7 @@ public void testMutate() throws IOException, ServiceException { rowCacheService.checkAndMutate(region, mutations, checkAndMutate, 0, 0); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRow(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -241,7 +241,7 @@ public void testMutate() throws IOException, ServiceException { rowCacheService.batchMutate(region, mutationArray, true, 0, 0); // Verify the sequence of method calls inOrder.verify(rowCacheService, Mockito.times(1)).createRowLevelBarrier(Mockito.any()); - inOrder.verify(rowCacheService, Mockito.times(1)).evictRowCache(Mockito.any()); + inOrder.verify(rowCacheService, Mockito.times(1)).evictRow(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).execute(Mockito.any()); inOrder.verify(rowCacheService, Mockito.times(1)).removeRowLevelBarrier(Mockito.any()); @@ -311,13 +311,13 @@ public void testCaching() throws IOException { // This should be called first not to populate the row cache get.setCacheBlocks(false); rowCacheService.getScanner(region, get, scan, results, context); - assertNull(rowCache.getBlock(key, true)); - assertNull(rowCache.getBlock(key, false)); + assertNull(rowCache.getRow(key, true)); + assertNull(rowCache.getRow(key, false)); // Verify that row cache populated with caching=true get.setCacheBlocks(true); rowCacheService.getScanner(region, get, scan, results, context); - assertNotNull(rowCache.getBlock(key, true)); - assertNull(rowCache.getBlock(key, false)); + assertNotNull(rowCache.getRow(key, true)); + assertNull(rowCache.getRow(key, false)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java index 7c9eb35f28f9..5cfae00998b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java @@ -171,12 +171,12 @@ public void testRowCache() throws Exception { Result result0 = table.get(get0); assertNotNull(result0); RowCacheKey keyPrev0 = new RowCacheKey(regions[0], get0.getRow()); - assertNotNull(rowCache.getBlock(keyPrev0, true)); + assertNotNull(rowCache.getRow(keyPrev0, true)); Get get1 = new Get(rowKeyRegion1); Result result1 = table.get(get1); assertNotNull(result1); RowCacheKey keyPrev1 = new RowCacheKey(regions[1], get1.getRow()); - assertNotNull(rowCache.getBlock(keyPrev1, true)); + assertNotNull(rowCache.getRow(keyPrev1, true)); // Do bulkload to region0 only Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); @@ -187,13 +187,13 @@ public void testRowCache() throws Exception { // Ensure the row cache is removed after bulkload for region0 RowCacheKey keyCur0 = new RowCacheKey(regions[0], get0.getRow()); assertNotEquals(keyPrev0, keyCur0); - assertNull(rowCache.getBlock(keyCur0, true)); + assertNull(rowCache.getRow(keyCur0, true)); // Ensure the row cache for keyPrev0 still exists, but it is not used anymore. - assertNotNull(rowCache.getBlock(keyPrev0, true)); + assertNotNull(rowCache.getRow(keyPrev0, true)); // Ensure the row cache for region1 is not affected RowCacheKey keyCur1 = new RowCacheKey(regions[1], get1.getRow()); assertEquals(keyPrev1, keyCur1); - assertNotNull(rowCache.getBlock(keyCur1, true)); + assertNotNull(rowCache.getRow(keyCur1, true)); } } From 859f6b090a0abf1c63cd5c44532eb47629073214 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Wed, 15 Oct 2025 15:44:26 +0900 Subject: [PATCH 13/21] Fix spotless violation --- .../java/org/apache/hadoop/hbase/regionserver/RowCache.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java index d7914a1788ff..2be7979e4001 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java @@ -56,8 +56,7 @@ public void onRemoval(RowCacheKey key, RowCells value, @NonNull RemovalCause cau Caffeine.newBuilder().maximumWeight(maxSizeBytes).removalListener(new EvictionListener()) .weigher((RowCacheKey key, RowCells value) -> (int) Math.min(key.heapSize() + value.heapSize(), Integer.MAX_VALUE)) - .recordStats() - .build(); + .recordStats().build(); } void cacheRow(RowCacheKey key, RowCells value) { From cceda4821812f42db111b7c06bc9576547ca0751 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Wed, 15 Oct 2025 15:51:52 +0900 Subject: [PATCH 14/21] Modify comment for region level invalidation --- .../main/java/org/apache/hadoop/hbase/regionserver/HRegion.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 7ef3e67a25a3..773350dc6e95 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -8708,7 +8708,7 @@ public long getRowCacheSeqNum() { } /** - * This is used to invalidate the entire row cache after bulk loading. + * This is used to invalidate the row cache of the bulk-loaded region. */ public void increaseRowCacheSeqNum() { this.rowCacheSeqNum.incrementAndGet(); From 6545e3196ea7259b9254dcee32d6db5322fae1e4 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Thu, 16 Oct 2025 10:34:41 +0900 Subject: [PATCH 15/21] Add ROW_CACHE_EVICT_ON_CLOSE configuration option --- .../org/apache/hadoop/hbase/HConstants.java | 6 + .../hadoop/hbase/regionserver/HRegion.java | 20 +++ .../hadoop/hbase/regionserver/RowCache.java | 9 ++ .../hbase/regionserver/RowCacheKey.java | 10 +- .../hbase/regionserver/RowCacheService.java | 4 + .../TestRowCacheEvictOnClose.java | 130 ++++++++++++++++++ 6 files changed, 176 insertions(+), 3 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheEvictOnClose.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 5381a4e3a18b..960762cff8a0 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1023,6 +1023,12 @@ public enum OperationStatusCode { public static final String ROW_CACHE_SIZE_KEY = "row.cache.size"; public static final float ROW_CACHE_SIZE_DEFAULT = 0.0f; + /** + * Configuration key for the evict the row cache on close + */ + public static final String ROW_CACHE_EVICT_ON_CLOSE_KEY = "row.cache.evictOnClose"; + public static final boolean ROW_CACHE_EVICT_ON_CLOSE_DEFAULT = false; + /** * Configuration key for the memory size of the block cache */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 773350dc6e95..79671bb12989 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_EVICT_ON_CLOSE_DEFAULT; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_EVICT_ON_CLOSE_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.REGION_NAMES_KEY; import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.ROW_LOCK_READ_LOCK_KEY; @@ -1946,6 +1948,8 @@ public Pair> call() throws IOException { } } + evictRowCache(); + status.setStatus("Writing region close event to WAL"); // Always write close marker to wal even for read only table. This is not a big problem as we // do not write any data into the region; it is just a meta edit in the WAL file. @@ -1986,6 +1990,22 @@ public Pair> call() throws IOException { } } + private void evictRowCache() { + boolean evictOnClose = getReadOnlyConfiguration().getBoolean(ROW_CACHE_EVICT_ON_CLOSE_KEY, + ROW_CACHE_EVICT_ON_CLOSE_DEFAULT); + + if (!evictOnClose) { + return; + } + + if (!(rsServices instanceof HRegionServer regionServer)) { + return; + } + + RowCacheService rowCacheService = regionServer.getRSRpcServices().getRowCacheService(); + rowCacheService.evictRowsByRegion(this); + } + /** Wait for all current flushes and compactions of the region to complete */ // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for // Phoenix needs. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java index 2be7979e4001..0b83d0d13e1b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java @@ -75,6 +75,15 @@ void evictRow(RowCacheKey key) { cache.asMap().remove(key); } + /** + * Evict all rows belonging to the specified region. This is heavy operation as it iterates the + * entire RowCache key set. + * @param region the region whose rows should be evicted + */ + void evictRowsByRegion(HRegion region) { + cache.asMap().keySet().removeIf(key -> key.isSameRegion(region)); + } + public long getHitCount() { return cache.stats().hitCount(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheKey.java index dc293ec8a053..09ec68194ea9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheKey.java @@ -30,9 +30,9 @@ public class RowCacheKey implements HeapSize { private final String encodedRegionName; private final byte[] rowKey; - // Row cache keys should not be evicted on close, since the cache may contain many entries and - // eviction would be slow. Instead, the region’s rowCacheSeqNum is used to generate new keys that - // ignore the existing cache when the region is reopened or bulk-loaded. + // When a region is reopened or bulk-loaded, its rowCacheSeqNum is used to generate new keys that + // bypass the existing cache. This mechanism is effective when ROW_CACHE_EVICT_ON_CLOSE is set to + // false. private final long rowCacheSeqNum; public RowCacheKey(HRegion region, byte[] rowKey) { @@ -64,4 +64,8 @@ public String toString() { public long heapSize() { return FIXED_OVERHEAD + ClassSize.align(rowKey.length); } + + boolean isSameRegion(HRegion region) { + return this.encodedRegionName.equals(region.getRegionInfo().getEncodedName()); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index 655ad4d646bc..7faa6743617a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -353,4 +353,8 @@ AtomicInteger getRegionLevelBarrier(HRegion region) { public RowCache getRowCache() { return rowCache; } + + void evictRowsByRegion(HRegion region) { + rowCache.evictRowsByRegion(region); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheEvictOnClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheEvictOnClose.java new file mode 100644 index 000000000000..e682a599d30d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheEvictOnClose.java @@ -0,0 +1,130 @@ +/* + * 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.regionserver; + +import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_EVICT_ON_CLOSE_KEY; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@Category({ RegionServerTests.class, MediumTests.class }) +@RunWith(Parameterized.class) +public class TestRowCacheEvictOnClose { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheEvictOnClose.class); + + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final byte[] CF1 = Bytes.toBytes("cf1"); + private static final byte[] Q1 = Bytes.toBytes("q1"); + private static final byte[] Q2 = Bytes.toBytes("q2"); + + @Rule + public TestName testName = new TestName(); + + @Parameterized.Parameter + public boolean evictOnClose; + + @Parameterized.Parameters + public static List params() { + return Arrays.asList(new Object[][] { { true }, { false } }); + } + + @Test + public void testEvictOnClose() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + + // Enable row cache + conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); + conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.39f); + + // Set ROW_CACHE_EVICT_ON_CLOSE + conf.setBoolean(ROW_CACHE_EVICT_ON_CLOSE_KEY, evictOnClose); + + // Start cluster + SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); + cluster.waitForActiveAndReadyMaster(); + Admin admin = TEST_UTIL.getAdmin(); + + RowCache rowCache = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRSRpcServices() + .getRowCacheService().getRowCache(); + + // Create table with row cache enabled + ColumnFamilyDescriptor cf1 = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + TableName tableName = TableName.valueOf(testName.getMethodName().replaceAll("[\\[\\]]", "_")); + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName).setRowCacheEnabled(true) + .setColumnFamily(cf1).build(); + admin.createTable(td); + Table table = admin.getConnection().getTable(tableName); + + int numRows = 10; + + // Put rows + for (int i = 0; i < numRows; i++) { + byte[] rowKey = ("row" + i).getBytes(); + Put put = new Put(rowKey); + put.addColumn(CF1, Q1, Bytes.toBytes(0L)); + put.addColumn(CF1, Q2, "12".getBytes()); + table.put(put); + } + // Need to flush because the row cache is not populated when reading only from the memstore. + admin.flush(tableName); + + // Populate row caches + for (int i = 0; i < numRows; i++) { + byte[] rowKey = ("row" + i).getBytes(); + Get get = new Get(rowKey); + Result result = table.get(get); + assertArrayEquals(rowKey, result.getRow()); + assertArrayEquals(Bytes.toBytes(0L), result.getValue(CF1, Q1)); + assertArrayEquals("12".getBytes(), result.getValue(CF1, Q2)); + } + + // Verify row cache has some entries + assertEquals(numRows, rowCache.getCount()); + + // Disable table + admin.disableTable(tableName); + + // Verify row cache is cleared on table close + assertEquals(evictOnClose ? 0 : numRows, rowCache.getCount()); + + admin.deleteTable(tableName); + TEST_UTIL.shutdownMiniCluster(); + } +} From 113445d64b7b99a953a1de6edb3f4fb862d376f5 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Thu, 16 Oct 2025 12:27:04 +0900 Subject: [PATCH 16/21] Add RowCache interface --- .../hadoop/hbase/regionserver/RowCache.java | 128 +++++++----------- .../hbase/regionserver/RowCacheService.java | 3 +- .../hbase/regionserver/TinyLfuRowCache.java | 116 ++++++++++++++++ 3 files changed, 170 insertions(+), 77 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TinyLfuRowCache.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java index 0b83d0d13e1b..a977c7d59cf2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCache.java @@ -17,96 +17,72 @@ */ package org.apache.hadoop.hbase.regionserver; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; -import com.github.benmanes.caffeine.cache.Policy; -import com.github.benmanes.caffeine.cache.RemovalCause; -import com.github.benmanes.caffeine.cache.RemovalListener; -import java.util.Optional; -import java.util.OptionalLong; -import java.util.concurrent.atomic.LongAdder; -import org.checkerframework.checker.nullness.qual.NonNull; - /** - * A cache that stores rows retrieved by Get operations, using Caffeine as the underlying cache - * implementation. + * Interface for caching rows retrieved by Get operations. */ @org.apache.yetus.audience.InterfaceAudience.Private -public class RowCache { - private final class EvictionListener - implements RemovalListener<@NonNull RowCacheKey, @NonNull RowCells> { - @Override - public void onRemoval(RowCacheKey key, RowCells value, @NonNull RemovalCause cause) { - evictedRowCount.increment(); - } - } - - private final Cache<@NonNull RowCacheKey, RowCells> cache; - - // Cache.stats() does not provide eviction count for entries, so we maintain our own counter. - private final LongAdder evictedRowCount = new LongAdder(); - - RowCache(long maxSizeBytes) { - if (maxSizeBytes <= 0) { - cache = Caffeine.newBuilder().maximumSize(0).build(); - return; - } - - cache = - Caffeine.newBuilder().maximumWeight(maxSizeBytes).removalListener(new EvictionListener()) - .weigher((RowCacheKey key, - RowCells value) -> (int) Math.min(key.heapSize() + value.heapSize(), Integer.MAX_VALUE)) - .recordStats().build(); - } - - void cacheRow(RowCacheKey key, RowCells value) { - cache.put(key, value); - } - - public RowCells getRow(RowCacheKey key, boolean caching) { - if (!caching) { - return null; - } - - return cache.getIfPresent(key); - } +public interface RowCache { + /** + * Cache the specified row. + * @param key the key of the row to cache + * @param value the cells of the row to cache + */ + void cacheRow(RowCacheKey key, RowCells value); - void evictRow(RowCacheKey key) { - cache.asMap().remove(key); - } + /** + * Evict the specified row. + * @param key the key of the row to evict + */ + void evictRow(RowCacheKey key); /** * Evict all rows belonging to the specified region. This is heavy operation as it iterates the * entire RowCache key set. * @param region the region whose rows should be evicted */ - void evictRowsByRegion(HRegion region) { - cache.asMap().keySet().removeIf(key -> key.isSameRegion(region)); - } + void evictRowsByRegion(HRegion region); - public long getHitCount() { - return cache.stats().hitCount(); - } + /** + * Get the number of rows in the cache. + * @return the number of rows in the cache + */ + long getCount(); - public long getMissCount() { - return cache.stats().missCount(); - } + /** + * Get the number of rows evicted from the cache. + * @return the number of rows evicted from the cache + */ + long getEvictedRowCount(); - public long getEvictedRowCount() { - return evictedRowCount.sum(); - } + /** + * Get the hit count. + * @return the hit count + */ + long getHitCount(); - public long getSize() { - Optional result = cache.policy().eviction().map(Policy.Eviction::weightedSize); - return result.orElse(OptionalLong.of(-1L)).orElse(-1L); - } + /** + * Get the maximum size of the cache in bytes. + * @return the maximum size of the cache in bytes + */ + long getMaxSize(); - public long getMaxSize() { - Optional result = cache.policy().eviction().map(Policy.Eviction::getMaximum); - return result.orElse(-1L); - } + /** + * Get the miss count. + * @return the miss count + */ + long getMissCount(); + + /** + * Get the specified row from the cache. + * @param key the key of the row to get + * @param caching whether caching is enabled for this request + * @return the cells of the row, or null if not found or caching is disabled + */ + RowCells getRow(RowCacheKey key, boolean caching); - public long getCount() { - return cache.estimatedSize(); - } + /** + * Get the current size of the cache in bytes. + * @return the current size of the cache in bytes + */ + long getSize(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index 7faa6743617a..25e2c412fbfb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -78,7 +78,8 @@ interface RowOperation { RowCacheService(Configuration conf) { enabledByConf = conf.getFloat(HConstants.ROW_CACHE_SIZE_KEY, HConstants.ROW_CACHE_SIZE_DEFAULT) > 0; - rowCache = new RowCache(MemorySizeUtil.getRowCacheSize(conf)); + // Currently we only support TinyLfu implementation + rowCache = new TinyLfuRowCache(MemorySizeUtil.getRowCacheSize(conf)); } RegionScannerImpl getScanner(HRegion region, Get get, Scan scan, List results, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TinyLfuRowCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TinyLfuRowCache.java new file mode 100644 index 000000000000..a23fd171c13d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TinyLfuRowCache.java @@ -0,0 +1,116 @@ +/* + * 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.regionserver; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Policy; +import com.github.benmanes.caffeine.cache.RemovalCause; +import com.github.benmanes.caffeine.cache.RemovalListener; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.atomic.LongAdder; +import org.checkerframework.checker.nullness.qual.NonNull; + +/** + * A {@link RowCache} implementation backed by Caffeine with a TinyLFU-based eviction policy. + */ +@org.apache.yetus.audience.InterfaceAudience.Private +public class TinyLfuRowCache implements RowCache { + private final class EvictionListener + implements RemovalListener<@NonNull RowCacheKey, @NonNull RowCells> { + @Override + public void onRemoval(RowCacheKey key, RowCells value, @NonNull RemovalCause cause) { + evictedRowCount.increment(); + } + } + + private final Cache<@NonNull RowCacheKey, RowCells> cache; + + // Cache.stats() does not provide eviction count for entries, so we maintain our own counter. + private final LongAdder evictedRowCount = new LongAdder(); + + TinyLfuRowCache(long maxSizeBytes) { + if (maxSizeBytes <= 0) { + cache = Caffeine.newBuilder().maximumSize(0).build(); + return; + } + + cache = + Caffeine.newBuilder().maximumWeight(maxSizeBytes).removalListener(new EvictionListener()) + .weigher((RowCacheKey key, + RowCells value) -> (int) Math.min(key.heapSize() + value.heapSize(), Integer.MAX_VALUE)) + .recordStats().build(); + } + + @Override + public void cacheRow(RowCacheKey key, RowCells value) { + cache.put(key, value); + } + + @Override + public RowCells getRow(RowCacheKey key, boolean caching) { + if (!caching) { + return null; + } + + return cache.getIfPresent(key); + } + + @Override + public void evictRow(RowCacheKey key) { + cache.asMap().remove(key); + } + + @Override + public void evictRowsByRegion(HRegion region) { + cache.asMap().keySet().removeIf(key -> key.isSameRegion(region)); + } + + @Override + public long getHitCount() { + return cache.stats().hitCount(); + } + + @Override + public long getMissCount() { + return cache.stats().missCount(); + } + + @Override + public long getEvictedRowCount() { + return evictedRowCount.sum(); + } + + @Override + public long getSize() { + Optional result = cache.policy().eviction().map(Policy.Eviction::weightedSize); + return result.orElse(OptionalLong.of(-1L)).orElse(-1L); + } + + @Override + public long getMaxSize() { + Optional result = cache.policy().eviction().map(Policy.Eviction::getMaximum); + return result.orElse(-1L); + } + + @Override + public long getCount() { + return cache.estimatedSize(); + } +} From 5dad0216c3e2df40107c430b85338e615945c215 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Thu, 16 Oct 2025 12:44:31 +0900 Subject: [PATCH 17/21] Minor fix --- .../java/org/apache/hadoop/hbase/regionserver/TestRowCache.java | 2 +- .../apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java index 7d1115e7d85a..af4fb14b75cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java @@ -104,7 +104,7 @@ public static void beforeClass() throws Exception { // Enable row cache but reduce the block cache size to fit in 80% of the heap conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); - conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); + conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.39f); SingleProcessHBaseCluster cluster = TEST_UTIL.startMiniCluster(); cluster.waitForActiveAndReadyMaster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java index 5cfae00998b6..22ae124fb9aa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesRowCache.java @@ -114,7 +114,7 @@ public static void setupCluster() throws Exception { // Enable row cache but reduce the block cache size to fit in 80% of the heap conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); - conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.38f); + conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.39f); TEST_UTIL.startMiniCluster(1); admin = TEST_UTIL.getAdmin(); From 4db38bd43b6e18fcffee1e71a46244eaab365020 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Mon, 20 Oct 2025 12:18:56 +0900 Subject: [PATCH 18/21] Support row cache when off heap cache is enabled --- .../io/encoding/BufferedDataBlockEncoder.java | 4 +- ...heWithBucketCacheAndDataBlockEncoding.java | 153 ++++++++++++++++++ 2 files changed, 155 insertions(+), 2 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheWithBucketCacheAndDataBlockEncoding.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java index c3d113c8b655..54505dfce955 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java @@ -796,8 +796,8 @@ public void write(ByteBuffer buf, int offset) { @Override public ExtendedCell deepClone() { - // This is not used in actual flow. Throwing UnsupportedOperationException - throw new UnsupportedOperationException(); + // To cache row, we need to deep clone it + return super.deepClone(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheWithBucketCacheAndDataBlockEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheWithBucketCacheAndDataBlockEncoding.java new file mode 100644 index 000000000000..6abf3ca8f4bf --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheWithBucketCacheAndDataBlockEncoding.java @@ -0,0 +1,153 @@ +/* + * 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.regionserver; + +import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; +import static org.apache.hadoop.hbase.HConstants.ROW_CACHE_SIZE_KEY; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.io.ByteBuffAllocator; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@Category(MediumTests.class) +@RunWith(Parameterized.class) +public class TestRowCacheWithBucketCacheAndDataBlockEncoding { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRowCacheWithBucketCacheAndDataBlockEncoding.class); + + @Parameterized.Parameter + public static boolean uesBucketCache; + + @Parameterized.Parameters + public static List params() { + return Arrays.asList(new Object[][] { { true }, { false } }); + } + + @Rule + public TestName name = new TestName(); + + private static final byte[] ROW_KEY = Bytes.toBytes("checkRow"); + private static final byte[] CF = Bytes.toBytes("CF"); + private static final byte[] QUALIFIER = Bytes.toBytes("cq"); + private static final byte[] VALUE = Bytes.toBytes("checkValue"); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final Configuration conf = TEST_UTIL.getConfiguration(); + private static Admin admin = null; + private static RowCache rowCache; + + @BeforeClass + public static void beforeClass() throws Exception { + // Use bucket cache + if (uesBucketCache) { + conf.setInt(ByteBuffAllocator.MIN_ALLOCATE_SIZE_KEY, 1); + conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap"); + conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 64); + } + + // Use row cache + conf.setFloat(ROW_CACHE_SIZE_KEY, 0.01f); + conf.setFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.39f); + TEST_UTIL.startMiniCluster(); + admin = TEST_UTIL.getAdmin(); + + rowCache = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRSRpcServices() + .getRowCacheService().getRowCache(); + } + + @AfterClass + public static void afterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testRowCacheNoEncode() throws Exception { + testRowCache(name.getMethodName(), DataBlockEncoding.NONE); + } + + @Test + public void testRowCacheEncode() throws Exception { + testRowCache(name.getMethodName(), DataBlockEncoding.FAST_DIFF); + } + + private void testRowCache(String methodName, DataBlockEncoding dbe) throws Exception { + TableName tableName = TableName.valueOf(methodName.replaceAll("[\\[\\]]", "_")); + try (Table testTable = createTable(tableName, dbe)) { + Put put = new Put(ROW_KEY); + put.addColumn(CF, QUALIFIER, VALUE); + testTable.put(put); + admin.flush(testTable.getName()); + + long countBase = rowCache.getCount(); + long hitCountBase = rowCache.getHitCount(); + + Result result; + + // First get should not hit the row cache, and populate it + Get get = new Get(ROW_KEY); + result = testTable.get(get); + assertArrayEquals(ROW_KEY, result.getRow()); + assertArrayEquals(VALUE, result.getValue(CF, QUALIFIER)); + assertEquals(1, rowCache.getCount() - countBase); + assertEquals(0, rowCache.getHitCount() - hitCountBase); + + // Second get should hit the row cache + result = testTable.get(get); + assertArrayEquals(ROW_KEY, result.getRow()); + assertArrayEquals(VALUE, result.getValue(CF, QUALIFIER)); + assertEquals(1, rowCache.getCount() - countBase); + assertEquals(1, rowCache.getHitCount() - hitCountBase); + } + } + + private Table createTable(TableName tableName, DataBlockEncoding dbe) throws IOException { + TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF).setBlocksize(100) + .setDataBlockEncoding(dbe).build()) + .setRowCacheEnabled(true).build(); + return TEST_UTIL.createTable(td, null); + } +} From 24a7ba8ebfea8a90159ba247a0df6ecacf57e4bf Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Tue, 21 Oct 2025 08:14:49 +0900 Subject: [PATCH 19/21] Add global row.cache.enabled configuration --- .../hadoop/hbase/client/TableDescriptor.java | 7 +- .../hbase/client/TableDescriptorBuilder.java | 6 +- .../org/apache/hadoop/hbase/HConstants.java | 6 ++ .../hadoop/hbase/regionserver/HRegion.java | 20 +++++ .../hbase/regionserver/RSRpcServices.java | 2 +- .../hadoop/hbase/regionserver/Region.java | 6 ++ .../hbase/regionserver/RowCacheService.java | 8 +- .../regionserver/TestRowCacheCanCacheRow.java | 7 +- .../TestRowCacheConfiguration.java | 81 +++++++++++++++++++ .../regionserver/TestRowCacheService.java | 6 +- 10 files changed, 135 insertions(+), 14 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfiguration.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java index 6d7786b5c88a..69871d9dc2fc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java @@ -318,9 +318,10 @@ default boolean matchReplicationScope(boolean enabled) { } /** - * Checks whether row caching is enabled for this table. Note that row caching applies only at the + * Checks whether row caching is enabled for this table. Note that row caching is applied at the * entire row level, not at the column family level. - * @return {@code true} if row cache is enabled, otherwise {@code false} + * @return {@code true} if row caching is enabled, {@code false} if disabled, or {@code null} if + * not explicitly set */ - boolean isRowCacheEnabled(); + Boolean getRowCacheEnabled(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index 64bd686b3360..99692b36fa4f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -245,6 +245,7 @@ public class TableDescriptorBuilder { DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY)); // Setting ERASURE_CODING_POLICY to NULL so that it is not considered as metadata DEFAULT_VALUES.put(ERASURE_CODING_POLICY, String.valueOf(DEFAULT_ERASURE_CODING_POLICY)); + DEFAULT_VALUES.put(ROW_CACHE_ENABLED, String.valueOf(DEFAULT_ROW_CACHE_ENABLED)); DEFAULT_VALUES.keySet().stream().map(s -> new Bytes(Bytes.toBytes(s))) .forEach(RESERVED_KEYWORDS::add); RESERVED_KEYWORDS.add(IS_META_KEY); @@ -1526,8 +1527,9 @@ public Optional getRegionServerGroup() { } @Override - public boolean isRowCacheEnabled() { - return getOrDefault(ROW_CACHE_ENABLED_KEY, Boolean::valueOf, DEFAULT_ROW_CACHE_ENABLED); + public Boolean getRowCacheEnabled() { + Bytes value = getValue(ROW_CACHE_ENABLED_KEY); + return value == null ? null : Boolean.valueOf(Bytes.toString(value.get())); } public ModifyableTableDescriptor setRowCacheEnabled(boolean enabled) { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 960762cff8a0..e3bb13e6ac8f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1029,6 +1029,12 @@ public enum OperationStatusCode { public static final String ROW_CACHE_EVICT_ON_CLOSE_KEY = "row.cache.evictOnClose"; public static final boolean ROW_CACHE_EVICT_ON_CLOSE_DEFAULT = false; + /** + * Configuration key for the row cache enabled + */ + public static final String ROW_CACHE_ENABLED_KEY = "row.cache.enabled"; + public static final boolean ROW_CACHE_ENABLED_DEFAULT = false; + /** * Configuration key for the memory size of the block cache */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 79671bb12989..264891033562 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -441,6 +441,11 @@ public MetricsTableRequests getMetricsTableRequests() { */ private final AtomicLong rowCacheSeqNum = new AtomicLong(HConstants.NO_SEQNUM); + /** + * The setting for whether to enable row cache for this region. + */ + private final boolean isRowCacheEnabled; + /** * The default setting for whether to enable on-demand CF loading for scan requests to this * region. Requests can override it. @@ -937,6 +942,16 @@ public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration co minBlockSizeBytes = Arrays.stream(this.htableDescriptor.getColumnFamilies()) .mapToInt(ColumnFamilyDescriptor::getBlocksize).min().orElse(HConstants.DEFAULT_BLOCKSIZE); + + this.isRowCacheEnabled = determineRowCacheEnabled(); + } + + boolean determineRowCacheEnabled() { + Boolean fromDescriptor = htableDescriptor.getRowCacheEnabled(); + // The setting from TableDescriptor has higher priority than the global configuration + return fromDescriptor != null + ? fromDescriptor + : conf.getBoolean(HConstants.ROW_CACHE_ENABLED_KEY, HConstants.ROW_CACHE_ENABLED_DEFAULT); } private void setHTableSpecificConf() { @@ -8727,6 +8742,11 @@ public long getRowCacheSeqNum() { return this.rowCacheSeqNum.get(); } + @Override + public boolean isRowCacheEnabled() { + return isRowCacheEnabled; + } + /** * This is used to invalidate the row cache of the bulk-loaded region. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index f170ed7d9062..72c42fd38d9f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -3043,7 +3043,7 @@ private void put(HRegion region, OperationQuota quota, MutationProto mutation, Put put = ProtobufUtil.toPut(mutation, cellScanner); // Put with TTL is not allowed on tables with row cache enabled, because cached rows cannot // track TTL expiration - if (region.getTableDescriptor().isRowCacheEnabled()) { + if (region.isRowCacheEnabled()) { if (put.getTTL() != Long.MAX_VALUE) { throw new DoNotRetryIOException( "Tables with row cache enabled do not allow setting TTL on Puts"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java index 42069e58092e..48d4add69ee6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java @@ -577,4 +577,10 @@ void requestCompaction(byte[] family, String why, int priority, boolean major, * estimating quota consumption. */ int getMinBlockSizeBytes(); + + /** + * Returns whether the row cache is enabled for this region. + * @return true if the row cache is enabled for this region + */ + boolean isRowCacheEnabled(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java index 25e2c412fbfb..f0549fdda54b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowCacheService.java @@ -154,7 +154,7 @@ BulkLoadHFileResponse bulkLoadHFile(RSRpcServices rsRpcServices, BulkLoadHFileRe throw new ServiceException(ie); } - if (!region.getTableDescriptor().isRowCacheEnabled()) { + if (!region.isRowCacheEnabled()) { return bulkLoad(rsRpcServices, request); } @@ -216,7 +216,7 @@ void createRegionLevelBarrier(HRegion region) { */ // @formatter:on boolean canCacheRow(Get get, Region region) { - return enabledByConf && region.getTableDescriptor().isRowCacheEnabled() && get.getCacheBlocks() + return enabledByConf && region.isRowCacheEnabled() && get.getCacheBlocks() && get.getFilter() == null && isRetrieveAllCells(get, region) && isDefaultTtl(region) && get.getAttributesMap().isEmpty() && !get.isCheckExistenceOnly() && get.getColumnFamilyTimeRange().isEmpty() && get.getConsistency() == Consistency.STRONG @@ -240,7 +240,7 @@ private static boolean isDefaultTtl(Region region) { private R mutateWithRowCacheBarrier(HRegion region, List mutations, RowOperation operation) throws IOException { - if (!region.getTableDescriptor().isRowCacheEnabled()) { + if (!region.isRowCacheEnabled()) { return operation.execute(); } @@ -267,7 +267,7 @@ private R mutateWithRowCacheBarrier(HRegion region, List mutations private R mutateWithRowCacheBarrier(HRegion region, byte[] row, RowOperation operation) throws IOException { - if (!region.getTableDescriptor().isRowCacheEnabled()) { + if (!region.isRowCacheEnabled()) { return operation.execute(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java index af1e8c5c3405..b774d6379f9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheCanCacheRow.java @@ -69,6 +69,7 @@ public void testRowCacheEnabledByTable() { Configuration conf = HBaseConfiguration.create(); conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + Mockito.when(region.isRowCacheEnabled()).thenReturn(td.getRowCacheEnabled()); RowCacheService rowCacheService = new RowCacheService(conf); Assert.assertTrue(rowCacheService.canCacheRow(get, region)); @@ -76,7 +77,7 @@ public void testRowCacheEnabledByTable() { // Disable row cache, expect false td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(cfd) .setRowCacheEnabled(false).build(); - Mockito.when(region.getTableDescriptor()).thenReturn(td); + Mockito.when(region.isRowCacheEnabled()).thenReturn(td.getRowCacheEnabled()); Assert.assertFalse(rowCacheService.canCacheRow(get, region)); } @@ -109,6 +110,7 @@ public void testRetrieveAllCells() { TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); + Mockito.when(region.isRowCacheEnabled()).thenReturn(td.getRowCacheEnabled()); Configuration conf = HBaseConfiguration.create(); conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); @@ -149,6 +151,7 @@ public void testTtl() { td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); + Mockito.when(region.isRowCacheEnabled()).thenReturn(td.getRowCacheEnabled()); Assert.assertFalse(rowCacheService.canCacheRow(get, region)); // Ttl is not set, expect true @@ -156,6 +159,7 @@ public void testTtl() { td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) .setColumnFamily(cfd1).setColumnFamily(cfd2).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); + Mockito.when(region.isRowCacheEnabled()).thenReturn(td.getRowCacheEnabled()); Assert.assertTrue(rowCacheService.canCacheRow(get, region)); } @@ -242,6 +246,7 @@ private static void testWith(Function func) { TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRowCacheEnabled(true) .setColumnFamily(cfd).build(); Mockito.when(region.getTableDescriptor()).thenReturn(td); + Mockito.when(region.isRowCacheEnabled()).thenReturn(td.getRowCacheEnabled()); Configuration conf = HBaseConfiguration.create(); conf.setFloat(HConstants.ROW_CACHE_SIZE_KEY, 0.01f); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfiguration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfiguration.java new file mode 100644 index 000000000000..4c70547d1812 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheConfiguration.java @@ -0,0 +1,81 @@ +/* + * 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.regionserver; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestRowCacheConfiguration { + private static final byte[] CF1 = "cf1".getBytes(); + private static final TableName TABLE_NAME = TableName.valueOf("table"); + private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + @Test + public void testDetermineRowCacheEnabled() throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + + HRegion region; + + // Set global config to false + conf.setBoolean(HConstants.ROW_CACHE_ENABLED_KEY, false); + + region = createRegion(null); + assertFalse(region.determineRowCacheEnabled()); + + region = createRegion(false); + assertFalse(region.determineRowCacheEnabled()); + + region = createRegion(true); + assertTrue(region.determineRowCacheEnabled()); + + // Set global config to true + conf.setBoolean(HConstants.ROW_CACHE_ENABLED_KEY, true); + + region = createRegion(null); + assertTrue(region.determineRowCacheEnabled()); + + region = createRegion(false); + assertFalse(region.determineRowCacheEnabled()); + + region = createRegion(true); + assertTrue(region.determineRowCacheEnabled()); + } + + private HRegion createRegion(Boolean rowCacheEnabled) throws IOException { + ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(CF1).build(); + TableDescriptorBuilder tdb = TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(cfd); + if (rowCacheEnabled != null) { + tdb.setRowCacheEnabled(rowCacheEnabled); + } + return TEST_UTIL.createLocalHRegion(tdb.build(), "".getBytes(), "1".getBytes()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java index 6e1c6c83a0e2..35977aadc790 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCacheService.java @@ -81,7 +81,6 @@ public void testBarrier() throws IOException { ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder("CF1".getBytes()).build(); TableDescriptor td = Mockito.mock(TableDescriptor.class); - Mockito.when(td.isRowCacheEnabled()).thenReturn(true); Mockito.when(td.getColumnFamilies()).thenReturn(new ColumnFamilyDescriptor[] { cfd }); byte[] rowKey = "row".getBytes(); @@ -98,6 +97,7 @@ public void testBarrier() throws IOException { Mockito.when(region.getStores()).thenReturn(stores); Mockito.when(region.getScanner(scan)).thenReturn(regionScanner); Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + Mockito.when(region.isRowCacheEnabled()).thenReturn(true); RpcCallContext context = Mockito.mock(RpcCallContext.class); Mockito.when(context.getBlockBytesScanned()).thenReturn(1L); @@ -157,7 +157,6 @@ public void testBarrier() throws IOException { public void testMutate() throws IOException, ServiceException { // Mocking RowCacheService and its dependencies TableDescriptor tableDescriptor = Mockito.mock(TableDescriptor.class); - Mockito.when(tableDescriptor.isRowCacheEnabled()).thenReturn(true); RegionInfo regionInfo = Mockito.mock(RegionInfo.class); Mockito.when(regionInfo.getEncodedName()).thenReturn("region1"); @@ -166,6 +165,7 @@ public void testMutate() throws IOException, ServiceException { Mockito.when(region.getTableDescriptor()).thenReturn(tableDescriptor); Mockito.when(region.getRegionInfo()).thenReturn(regionInfo); Mockito.when(region.getBlockCache()).thenReturn(Mockito.mock(BlockCache.class)); + Mockito.when(region.isRowCacheEnabled()).thenReturn(true); RSRpcServices rsRpcServices = Mockito.mock(RSRpcServices.class); Mockito.when(rsRpcServices.getRegion(Mockito.any())).thenReturn(region); @@ -278,7 +278,6 @@ public void testCaching() throws IOException { ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder("CF1".getBytes()).build(); TableDescriptor td = Mockito.mock(TableDescriptor.class); - Mockito.when(td.isRowCacheEnabled()).thenReturn(true); Mockito.when(td.getColumnFamilies()).thenReturn(new ColumnFamilyDescriptor[] { cfd }); RpcCallContext context = Mockito.mock(RpcCallContext.class); @@ -298,6 +297,7 @@ public void testCaching() throws IOException { Mockito.when(region.getStores()).thenReturn(stores); Mockito.when(region.getScanner(scan)).thenReturn(regionScanner); Mockito.when(region.getReadOnlyConfiguration()).thenReturn(conf); + Mockito.when(region.isRowCacheEnabled()).thenReturn(true); RowCacheKey key = new RowCacheKey(region, rowKey); List results = new ArrayList<>(); From 7e2718ae1a0c9bef2b6c3232ac5537c7aa945033 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Tue, 21 Oct 2025 11:54:13 +0900 Subject: [PATCH 20/21] Enhance TestRowCache to ensure eviction count increments only for actually evicted rows --- .../hbase/regionserver/TestRowCache.java | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java index af4fb14b75cc..48138a854052 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowCache.java @@ -175,15 +175,6 @@ public void testGetWithRowCache() throws IOException { RowCacheKey rowCacheKey = new RowCacheKey(region, rowKey); - // Put a row - Put put = new Put(rowKey); - put.addColumn(CF1, Q1, Bytes.toBytes(0L)); - put.addColumn(CF1, Q2, "12".getBytes()); - put.addColumn(CF2, Q1, "21".getBytes()); - put.addColumn(CF2, Q2, "22".getBytes()); - table.put(put); - admin.flush(tableName); - // Initialize metrics recomputeMetrics(); setCounterBase("Get_num_ops", metricsHelper.getCounter("Get_num_ops", serverSource)); @@ -194,6 +185,19 @@ public void testGetWithRowCache() throws IOException { setCounterBase(ROW_CACHE_EVICTED_ROW_COUNT, metricsHelper.getCounter(ROW_CACHE_EVICTED_ROW_COUNT, serverSource)); + // Put a row + Put put = new Put(rowKey); + put.addColumn(CF1, Q1, Bytes.toBytes(0L)); + put.addColumn(CF1, Q2, "12".getBytes()); + put.addColumn(CF2, Q1, "21".getBytes()); + put.addColumn(CF2, Q2, "22".getBytes()); + table.put(put); + admin.flush(tableName); + recomputeMetrics(); + assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); + assertCounterDiff(ROW_CACHE_MISS_COUNT, 0); + assertCounterDiff(ROW_CACHE_EVICTED_ROW_COUNT, 0); + // First get to populate the row cache result = table.get(get); recomputeMetrics(); @@ -206,6 +210,7 @@ public void testGetWithRowCache() throws IOException { // Ensure the get operation from HFile without row cache assertCounterDiff(ROW_CACHE_HIT_COUNT, 0); assertCounterDiff(ROW_CACHE_MISS_COUNT, 1); + assertCounterDiff(ROW_CACHE_EVICTED_ROW_COUNT, 0); // Get from the row cache result = table.get(get); @@ -219,6 +224,7 @@ public void testGetWithRowCache() throws IOException { // Ensure the get operation from the row cache assertCounterDiff(ROW_CACHE_HIT_COUNT, 1); assertCounterDiff(ROW_CACHE_MISS_COUNT, 0); + assertCounterDiff(ROW_CACHE_EVICTED_ROW_COUNT, 0); // Row cache is invalidated by the put operation assertNotNull(rowCache.getRow(rowCacheKey, true)); From 7b4ddd9ef736b7e0e2b19b5ca040fa3a62411206 Mon Sep 17 00:00:00 2001 From: "terence.yoo" Date: Wed, 22 Oct 2025 09:33:36 +0900 Subject: [PATCH 21/21] Fix incorrect changes from previous merge --- .../apache/hadoop/hbase/regionserver/http/TestRSStatusPage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/http/TestRSStatusPage.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/http/TestRSStatusPage.java index b4b7214b037d..fabc12ee835c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/http/TestRSStatusPage.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/http/TestRSStatusPage.java @@ -132,7 +132,7 @@ public void testStatusPage() throws Exception { assertTrue(page.contains(expectedPageHeader)); assertTrue(page.contains("

Server Metrics

")); assertTrue(page.contains("Requests Per Second")); - assertTrue(page.contains("

Block Cache

")); + assertTrue(page.contains("

Cache

")); assertTrue(page.contains("

Regions

")); assertTrue(page.contains("

Replication Status

")); assertTrue(page.contains("

Software Attributes

"));