You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2022/09/16 17:51:27 UTC

[GitHub] [hbase] taklwu commented on a diff in pull request #4781: HBASE-27370 Avoid decompressing blocks when reading from bucket cache…

taklwu commented on code in PR #4781:
URL: https://github.com/apache/hbase/pull/4781#discussion_r973246512


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -148,36 +153,88 @@ private void readStoreFileLikeScanner(Path storeFilePath) throws Exception {
   }
 
   private void readStoreFile(Path storeFilePath) throws Exception {
+    readStoreFile(storeFilePath, (r, o) -> {
+      HFileBlock block = null;
+      try {
+        block = r.readBlock(o, -1, false, true, false, true, null, null);
+      } catch (IOException e) {
+        fail(e.getMessage());
+      }
+      return block;
+    }, (key, block) -> {
+      boolean isCached = blockCache.getBlock(key, true, false, true) != null;
+      if (
+        block.getBlockType() == BlockType.DATA || block.getBlockType() == BlockType.ROOT_INDEX
+          || block.getBlockType() == BlockType.INTERMEDIATE_INDEX
+      ) {
+        assertTrue(isCached);
+      }
+    });
+  }
+
+  private void readStoreFileCacheOnly(Path storeFilePath) throws Exception {
+    readStoreFile(storeFilePath, (r, o) -> {
+      HFileBlock block = null;
+      try {
+        block = r.readBlock(o, -1, false, true, false, true, null, null, true);
+      } catch (IOException e) {
+        fail(e.getMessage());
+      }
+      return block;
+    }, (key, block) -> {
+      boolean isCached = blockCache.getBlock(key, true, false, true) != null;
+      if (block.getBlockType() == BlockType.DATA) {
+        assertFalse(block.isUnpacked());
+      } else if (
+        block.getBlockType() == BlockType.ROOT_INDEX
+          || block.getBlockType() == BlockType.INTERMEDIATE_INDEX
+      ) {
+        assertTrue(block.isUnpacked());
+      }
+      assertTrue(isCached);
+    });
+  }
+
+  private void readStoreFile(Path storeFilePath,
+    BiFunction<HFile.Reader, Long, HFileBlock> readFunction,
+    BiConsumer<BlockCacheKey, HFileBlock> validationFunction) throws Exception {
     // Open the file
     HFile.Reader reader = HFile.createReader(fs, storeFilePath, cacheConf, true, conf);
 
     while (!reader.prefetchComplete()) {
       // Sleep for a bit
       Thread.sleep(1000);
     }
-
-    // Check that all of the data blocks were preloaded
-    BlockCache blockCache = cacheConf.getBlockCache().get();
     long offset = 0;
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
-      HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null);
+      HFileBlock block = readFunction.apply(reader, offset);
       BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(), offset);
-      boolean isCached = blockCache.getBlock(blockCacheKey, true, false, true) != null;
-      if (
-        block.getBlockType() == BlockType.DATA || block.getBlockType() == BlockType.ROOT_INDEX
-          || block.getBlockType() == BlockType.INTERMEDIATE_INDEX
-      ) {
-        assertTrue(isCached);
-      }
+      validationFunction.accept(blockCacheKey, block);
       offset += block.getOnDiskSizeWithHeader();
     }
   }
 
+  @Test
+  public void testPrefetchCompressed() throws Exception {
+    conf.setBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, true);
+    cacheConf = new CacheConfig(conf, blockCache);
+    HFileContext context = new HFileContextBuilder().withCompression(Compression.Algorithm.GZ)
+      .withBlockSize(DATA_BLOCK_SIZE).build();
+    Path storeFile = writeStoreFile("TestPrefetchCompressed", context);
+    readStoreFileCacheOnly(storeFile);
+    conf.setBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, false);

Review Comment:
   [nit] is it for resetting configuration to avoid error for other tests? should we use a cleanup? but it's not a blocker



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java:
##########
@@ -1236,6 +1236,15 @@ private boolean shouldUseHeap(BlockType expectedBlockType) {
   public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, final boolean cacheBlock,
     boolean pread, final boolean isCompaction, boolean updateCacheMetrics,
     BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding) throws IOException {
+    return readBlock(dataBlockOffset, onDiskBlockSize, cacheBlock, pread, isCompaction,
+      updateCacheMetrics, expectedBlockType, expectedDataBlockEncoding, false);
+  }
+
+  @Override
+  public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, final boolean cacheBlock,
+    boolean pread, final boolean isCompaction, boolean updateCacheMetrics,
+    BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding, boolean cacheOnly)

Review Comment:
   [nit] this new flag `cacheOnly=true` should just skip reading blocks from the local cache, and it does not have any caller other in the unit tests, are you planning to introduce a new behavior in the future ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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