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

wchevreuil pushed a commit to branch branch-2.6
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.6 by this push:
     new f6642484248 HBASE-29243 Fix BucketCache.notifyFileCachingComplete to 
also account for ENCODED_DATA block type (#6885) (#6899)
f6642484248 is described below

commit f66424842480b2dc9e6be9ac52ff1047367dc592
Author: Wellington Ramos Chevreuil <[email protected]>
AuthorDate: Thu Apr 10 18:07:55 2025 +0100

    HBASE-29243 Fix BucketCache.notifyFileCachingComplete to also account for 
ENCODED_DATA block type (#6885) (#6899)
    
    Reviewed-by: Rahul Agarkar <[email protected]>
    Signed-off-by: Peter Somogyi <[email protected]>
---
 .../hadoop/hbase/io/hfile/bucket/BucketCache.java  | 20 ++++++-------
 .../hadoop/hbase/io/hfile/CacheTestUtils.java      | 24 ++++++++++------
 .../hbase/io/hfile/bucket/TestBucketCache.java     | 33 ++++++++++++++++++----
 3 files changed, 52 insertions(+), 25 deletions(-)

diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index d651f055bb7..f4789994b43 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -2179,7 +2179,7 @@ public class BucketCache implements BlockCache, HeapSize {
       backingMap.entrySet().stream().forEach(entry -> {
         if (
           entry.getKey().getHfileName().equals(fileName.getName())
-            && entry.getKey().getBlockType().equals(BlockType.DATA)
+            && entry.getKey().getBlockType().isData()
         ) {
           long offsetToLock = entry.getValue().offset();
           LOG.debug("found block {} in the backing map. Acquiring read lock 
for offset {}",
@@ -2197,7 +2197,6 @@ public class BucketCache implements BlockCache, HeapSize {
           }
         }
       });
-      int metaCount = totalBlockCount - dataBlockCount;
       // BucketCache would only have data blocks
       if (dataBlockCount == count.getValue()) {
         LOG.debug("File {} has now been fully cached.", fileName);
@@ -2217,17 +2216,14 @@ public class BucketCache implements BlockCache, 
HeapSize {
             + "and try the verification again.", fileName.getName());
           Thread.sleep(100);
           notifyFileCachingCompleted(fileName, totalBlockCount, 
dataBlockCount, size);
-        } else
-          if ((getAllCacheKeysForFile(fileName.getName()).size() - metaCount) 
== dataBlockCount) {
-            LOG.debug("We counted {} data blocks, expected was {}, there was 
no more pending in "
-              + "the cache write queue but we now found that total cached 
blocks for file {} "
-              + "is equal to data block count.", count, dataBlockCount, 
fileName.getName());
-            fileCacheCompleted(fileName, size);
-          } else {
-            LOG.info("We found only {} data blocks cached from a total of {} 
for file {}, "
+        } else {
+          LOG.info(
+            "The total block count was {}. We found only {} data blocks cached 
from "
+              + "a total of {} data blocks for file {}, "
               + "but no blocks pending caching. Maybe cache is full or 
evictions "
-              + "happened concurrently to cache prefetch.", count, 
dataBlockCount, fileName);
-          }
+              + "happened concurrently to cache prefetch.",
+            totalBlockCount, count, dataBlockCount, fileName);
+        }
       }
     } catch (InterruptedException e) {
       throw new RuntimeException(e);
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
index b8c9794665f..e66dd345418 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
@@ -40,6 +40,7 @@ import 
org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
 import org.apache.hadoop.hbase.io.ByteBuffAllocator;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -282,7 +283,8 @@ public class CacheTestUtils {
     return generateBlocksForPath(blockSize, numBlocks, null);
   }
 
-  public static HFileBlockPair[] generateBlocksForPath(int blockSize, int 
numBlocks, Path path) {
+  public static HFileBlockPair[] generateBlocksForPath(int blockSize, int 
numBlocks, Path path,
+    boolean encoded) {
     HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
     Random rand = ThreadLocalRandom.current();
     HashSet<String> usedStrings = new HashSet<>();
@@ -301,12 +303,13 @@ public class CacheTestUtils {
       HFileContext meta =
         new 
HFileContextBuilder().withHBaseCheckSum(false).withIncludesMvcc(includesMemstoreTS)
           .withIncludesTags(false).withCompression(Compression.Algorithm.NONE)
-          .withBytesPerCheckSum(0).withChecksumType(ChecksumType.NULL).build();
-      HFileBlock generated =
-        new HFileBlock(BlockType.DATA, onDiskSizeWithoutHeader, 
uncompressedSizeWithoutHeader,
-          prevBlockOffset, ByteBuff.wrap(cachedBuffer), 
HFileBlock.DONT_FILL_HEADER, blockSize,
-          onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, 
meta,
-          ByteBuffAllocator.HEAP);
+          
.withDataBlockEncoding(DataBlockEncoding.FAST_DIFF).withBytesPerCheckSum(0)
+          .withChecksumType(ChecksumType.NULL).build();
+      HFileBlock generated = new HFileBlock(encoded ? BlockType.ENCODED_DATA : 
BlockType.DATA,
+        onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader, 
prevBlockOffset,
+        ByteBuff.wrap(cachedBuffer), HFileBlock.DONT_FILL_HEADER, blockSize,
+        onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta,
+        ByteBuffAllocator.HEAP);
       String key = null;
       long offset = 0;
       if (path != null) {
@@ -320,12 +323,17 @@ public class CacheTestUtils {
         }
       }
       returnedBlocks[i] = new HFileBlockPair();
-      returnedBlocks[i].blockName = new BlockCacheKey(key, offset);
+      returnedBlocks[i].blockName =
+        new BlockCacheKey(key, offset, true, encoded ? BlockType.ENCODED_DATA 
: BlockType.DATA);
       returnedBlocks[i].block = generated;
     }
     return returnedBlocks;
   }
 
+  public static HFileBlockPair[] generateBlocksForPath(int blockSize, int 
numBlocks, Path path) {
+    return generateBlocksForPath(blockSize, numBlocks, path, false);
+  }
+
   public static class HFileBlockPair {
     BlockCacheKey blockName;
     HFileBlock block;
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
index 3c5a94a3344..41bd1d0fcd9 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import static 
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.ACCEPT_FACTOR_CONFIG_NAME;
 import static 
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BLOCK_ORPHAN_GRACE_PERIOD;
-import static 
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.DEFAULT_ERROR_TOLERATION_DURATION;
 import static 
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.EXTRA_FREE_FACTOR_CONFIG_NAME;
 import static 
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.MIN_FACTOR_CONFIG_NAME;
 import static 
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.QUEUE_ADDITION_WAIT_TIME;
@@ -913,7 +912,31 @@ public class TestBucketCache {
     try {
       Path filePath =
         new Path(HBASE_TESTING_UTILITY.getDataTestDir(), 
"testNotifyFileCachingCompletedSuccess");
-      bucketCache = testNotifyFileCachingCompletedForTenBlocks(filePath, 10);
+      bucketCache = testNotifyFileCachingCompletedForTenBlocks(filePath, 10, 
false);
+      if (bucketCache.getStats().getFailedInserts() > 0) {
+        LOG.info("There were {} fail inserts, "
+          + "will assert if total blocks in backingMap equals (10 - 
failInserts) "
+          + "and file isn't listed as fully cached.", 
bucketCache.getStats().getFailedInserts());
+        assertEquals(10 - bucketCache.getStats().getFailedInserts(), 
bucketCache.backingMap.size());
+        
assertFalse(bucketCache.fullyCachedFiles.containsKey(filePath.getName()));
+      } else {
+        
assertTrue(bucketCache.fullyCachedFiles.containsKey(filePath.getName()));
+      }
+    } finally {
+      if (bucketCache != null) {
+        bucketCache.shutdown();
+      }
+      HBASE_TESTING_UTILITY.cleanupTestDir();
+    }
+  }
+
+  @Test
+  public void testNotifyFileCachingCompletedForEncodedDataSuccess() throws 
Exception {
+    BucketCache bucketCache = null;
+    try {
+      Path filePath = new Path(HBASE_TESTING_UTILITY.getDataTestDir(),
+        "testNotifyFileCachingCompletedForEncodedDataSuccess");
+      bucketCache = testNotifyFileCachingCompletedForTenBlocks(filePath, 10, 
true);
       if (bucketCache.getStats().getFailedInserts() > 0) {
         LOG.info("There were {} fail inserts, "
           + "will assert if total blocks in backingMap equals (10 - 
failInserts) "
@@ -939,7 +962,7 @@ public class TestBucketCache {
         "testNotifyFileCachingCompletedNotAllCached");
       // Deliberately passing more blocks than we have created to test that
       // notifyFileCachingCompleted will not consider the file fully cached
-      bucketCache = testNotifyFileCachingCompletedForTenBlocks(filePath, 12);
+      bucketCache = testNotifyFileCachingCompletedForTenBlocks(filePath, 12, 
false);
       
assertFalse(bucketCache.fullyCachedFiles.containsKey(filePath.getName()));
     } finally {
       if (bucketCache != null) {
@@ -950,7 +973,7 @@ public class TestBucketCache {
   }
 
   private BucketCache testNotifyFileCachingCompletedForTenBlocks(Path filePath,
-    int totalBlocksToCheck) throws Exception {
+    int totalBlocksToCheck, boolean encoded) throws Exception {
     final Path dataTestDir = createAndGetTestDir();
     String ioEngineName = "file:" + dataTestDir + "/bucketNoRecycler.cache";
     BucketCache bucketCache = new BucketCache(ioEngineName, capacitySize, 
constructedBlockSize,
@@ -958,7 +981,7 @@ public class TestBucketCache {
     long usedByteSize = bucketCache.getAllocator().getUsedSize();
     assertEquals(0, usedByteSize);
     HFileBlockPair[] hfileBlockPairs =
-      CacheTestUtils.generateBlocksForPath(constructedBlockSize, 10, filePath);
+      CacheTestUtils.generateBlocksForPath(constructedBlockSize, 10, filePath, 
encoded);
     // Add blocks
     for (HFileBlockPair hfileBlockPair : hfileBlockPairs) {
       bucketCache.cacheBlock(hfileBlockPair.getBlockName(), 
hfileBlockPair.getBlock(), false, true);

Reply via email to