You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2021/10/19 07:52:02 UTC

[hbase] branch branch-2 updated: HBASE-26190 High rate logging of BucketAllocatorException: Allocation too big (#3752)

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 7e574c9  HBASE-26190 High rate logging of BucketAllocatorException: Allocation too big (#3752)
7e574c9 is described below

commit 7e574c9f1d2be6d0b861e4a5d1ce429837305309
Author: KulwantSingh011 <87...@users.noreply.github.com>
AuthorDate: Tue Oct 19 13:21:11 2021 +0530

    HBASE-26190 High rate logging of BucketAllocatorException: Allocation too big (#3752)
    
    Reduce the frequency of allocation failed traces by printing them
    preiodically (once per minute). Record the allocation failures in the
    Bucket Cache Stats and let the stat thread dump cumulative allocation
    failures alongside other traces it dumps.
    
    Also, this change adds trace for the Table name, Column Family and
    HFileName for the most recent allocation failure in last 1 minute.
    
    Signed-off-by: Anoop <an...@apache.org>
---
 .../apache/hadoop/hbase/io/hfile/HFileBlock.java   |  2 +-
 .../hadoop/hbase/io/hfile/bucket/BucketCache.java  | 37 ++++++++++++++++++++--
 .../hbase/io/hfile/bucket/BucketCacheStats.java    | 17 +++++++++-
 3 files changed, 52 insertions(+), 4 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index 3547356..70c5c8a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -2030,7 +2030,7 @@ public class HFileBlock implements Cacheable {
    * @return This HFileBlocks fileContext which will a derivative of the
    *   fileContext for the file from which this block's data was originally read.
    */
-  HFileContext getHFileContext() {
+  public HFileContext getHFileContext() {
     return this.fileContext;
   }
 
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 de7abd2..6f624c0 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
@@ -67,9 +67,11 @@ import org.apache.hadoop.hbase.io.hfile.CacheStats;
 import org.apache.hadoop.hbase.io.hfile.Cacheable;
 import org.apache.hadoop.hbase.io.hfile.CachedBlock;
 import org.apache.hadoop.hbase.io.hfile.HFileBlock;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.RefCnt;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.IdReadWriteLock;
 import org.apache.hadoop.hbase.util.IdReadWriteLock.ReferenceType;
@@ -244,6 +246,10 @@ public class BucketCache implements BlockCache, HeapSize {
    * */
   private String algorithm;
 
+  /* Tracing failed Bucket Cache allocations. */
+  private long allocFailLogPrevTs; // time of previous log event for allocation failure.
+  private static final int ALLOCATION_FAIL_LOG_TIME_PERIOD = 60000; // Default 1 minute.
+
   public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
       int writerThreadNum, int writerQLen, String persistencePath) throws IOException {
     this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
@@ -280,6 +286,8 @@ public class BucketCache implements BlockCache, HeapSize {
     this.blockSize = blockSize;
     this.ioErrorsTolerationDuration = ioErrorsTolerationDuration;
 
+    this.allocFailLogPrevTs = 0;
+
     bucketAllocator = new BucketAllocator(capacity, bucketSizes);
     for (int i = 0; i < writerThreads.length; ++i) {
       writerQueues.add(new ArrayBlockingQueue<>(writerQLen));
@@ -716,7 +724,8 @@ public class BucketCache implements BlockCache, HeapSize {
           (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
         "evictions=" + cacheStats.getEvictionCount() + ", " +
         "evicted=" + cacheStats.getEvictedCount() + ", " +
-        "evictedPerRun=" + cacheStats.evictedPerEviction());
+        "evictedPerRun=" + cacheStats.evictedPerEviction() + ", " +
+        "allocationFailCount=" + cacheStats.getAllocationFailCount());
     cacheStats.reset();
   }
 
@@ -985,6 +994,25 @@ public class BucketCache implements BlockCache, HeapSize {
   }
 
   /**
+   * Prepare and return a warning message for Bucket Allocator Exception
+   * @param re The RAMQueueEntry for which the exception was thrown.
+   * @return A warning message created from the input RAMQueueEntry object.
+   */
+  private String getAllocationFailWarningMessage(RAMQueueEntry re) {
+    if (re != null && re.getData() instanceof HFileBlock) {
+      HFileBlock block = (HFileBlock) re.getData();
+      HFileContext fileContext = block.getHFileContext();
+      String hFileName = fileContext.getHFileName();
+      String columnFamily = Bytes.toString(fileContext.getColumnFamily());
+      String tableName = Bytes.toString(fileContext.getTableName());
+      return ("Most recent failed allocation in " + ALLOCATION_FAIL_LOG_TIME_PERIOD
+              + " milliseconds; Table Name = " + tableName + ", Column Family = "
+              + columnFamily + ", HFile Name : " + hFileName);
+    }
+    return ("Failed allocation for " + (re == null ? "" : re.getKey()) + "; ");
+  }
+
+  /**
    * Flush the entries in ramCache to IOEngine and add bucket entry to backingMap. Process all that
    * are passed in even if failure being sure to remove from ramCache else we'll never undo the
    * references and we'll OOME.
@@ -1029,7 +1057,12 @@ public class BucketCache implements BlockCache, HeapSize {
         }
         index++;
       } catch (BucketAllocatorException fle) {
-        LOG.warn("Failed allocation for " + (re == null ? "" : re.getKey()) + "; " + fle);
+        long currTs = System.currentTimeMillis(); // Current time since Epoch in milliseconds.
+        cacheStats.allocationFailed(); // Record the warning.
+        if (allocFailLogPrevTs == 0 || (currTs - allocFailLogPrevTs) > ALLOCATION_FAIL_LOG_TIME_PERIOD) {
+          LOG.warn (getAllocationFailWarningMessage(re), fle);
+          allocFailLogPrevTs = currTs;
+        }
         // Presume can't add. Too big? Move index on. Entry will be cleared from ramCache below.
         bucketEntries[index] = null;
         index++;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
index 49b9bba..4a2b0a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
@@ -35,14 +35,20 @@ public class BucketCacheStats extends CacheStats {
   private static final long NANO_TIME = TimeUnit.MILLISECONDS.toNanos(1);
   private long lastLogTime = EnvironmentEdgeManager.currentTime();
 
+  /* Tracing failed Bucket Cache allocations. */
+  private LongAdder allocationFailCount = new LongAdder();
+
   BucketCacheStats() {
     super("BucketCache");
+
+    allocationFailCount.reset();
   }
 
   @Override
   public String toString() {
     return super.toString() + ", ioHitsPerSecond=" + getIOHitsPerSecond() +
-      ", ioTimePerHit=" + getIOTimePerHit();
+      ", ioTimePerHit=" + getIOTimePerHit() + ", allocationFailCount=" +
+      getAllocationFailCount();
   }
 
   public void ioHit(long time) {
@@ -66,5 +72,14 @@ public class BucketCacheStats extends CacheStats {
   public void reset() {
     ioHitCount.reset();
     ioHitTime.reset();
+    allocationFailCount.reset();
+  }
+
+  public long getAllocationFailCount() {
+    return allocationFailCount.sum();
+  }
+
+  public void allocationFailed () {
+    allocationFailCount.increment();
   }
 }