You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2013/04/23 04:49:32 UTC

svn commit: r1470790 - in /hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase: io/hfile/AbstractHFileWriter.java io/hfile/HFile.java io/hfile/HFileWriterV2.java regionserver/Store.java regionserver/metrics/RegionServerMetrics.java

Author: liyin
Date: Tue Apr 23 02:49:31 2013
New Revision: 1470790

URL: http://svn.apache.org/r1470790
Log:
[HBASE-5263] Change the logging to trace in HFileWriterV2 for cacheCurrentBlockForCompaction

Author: rshroff

Summary:
The logging level is debug by default on all the HBase clusters. The current
function cacheCurrentBlockForCompaction dumps the log at debug level for every
new block written during compaction.

Changing the level to trace to avoid spamming the log.

Test Plan: build and test it on TSH25

Reviewers: liyintang, nspiegelberg

Reviewed By: liyintang

CC: hbase-eng@

Differential Revision: https://phabricator.fb.com/D772585

Task ID: 2016630

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java?rev=1470790&r1=1470789&r2=1470790&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java Tue Apr 23 02:49:31 2013
@@ -127,6 +127,13 @@ public abstract class AbstractHFileWrite
 
   protected boolean isCompactionWriter = false;
 
+  protected int numBlocksCachedPerCompaction = 0;
+
+  @Override
+  public int getNumBlocksCachedPerCompaction() {
+    return numBlocksCachedPerCompaction;
+  }
+
   public AbstractHFileWriter(Configuration conf, CacheConfig cacheConf,
       FSDataOutputStream outputStream, Path path, int blockSize,
       Compression.Algorithm compressAlgo,

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1470790&r1=1470789&r2=1470790&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Tue Apr 23 02:49:31 2013
@@ -164,6 +164,7 @@ public class HFile {
 
   static final AtomicInteger writeOps = new AtomicInteger();
   static final AtomicLong writeTimeNano = new AtomicLong();
+  static final AtomicInteger blocksCachedDuringCompaction = new AtomicInteger();
 
   // For measuring latency of pread
   static final AtomicInteger preadOps = new AtomicInteger();
@@ -190,6 +191,13 @@ public class HFile {
   }
 
   /**
+   * Get number of blocks cached during compaction and reset the counter
+   */
+  public static final int getBlocksCachedDuringCompactionAndReset() {
+    return blocksCachedDuringCompaction.getAndSet(0);
+  }
+
+  /**
    * Get the total time of positional reads during compaction in milliseconds and reset it to zero.
    */
   public static final long getPreadCompactionTimeMsAndReset() {
@@ -277,6 +285,11 @@ public class HFile {
      * Set whether compaction is in progress or not
      */
     void setCompactionWriter(boolean isCompaction);
+
+    /**
+     * Returns the number of blocks cached during compaction
+     */
+    int getNumBlocksCachedPerCompaction();
   }
 
   /**

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java?rev=1470790&r1=1470789&r2=1470790&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java Tue Apr 23 02:49:31 2013
@@ -176,13 +176,16 @@ public class HFileWriterV2 extends Abstr
     float blockCachingPercentage =
         ((float)this.numCachedKeysInCurrentBlock)/this.numKeysInCurrentBlock;
 
-    LOG.debug("Block Caching %: " + blockCachingPercentage +
-        " CachingOnCompaction: " + this.cacheConf.shouldCacheOnCompaction() +
-        " Threshold: " + this.cacheConf.getCacheOnCompactionThreshold());
-
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Block Caching %: " + blockCachingPercentage +
+          " CachingOnCompaction: " + this.cacheConf.shouldCacheOnCompaction() +
+          " Threshold: " + this.cacheConf.getCacheOnCompactionThreshold());
+    }
     // Get the threshold from the config
     if (this.cacheConf.shouldCacheOnCompaction() &&
         blockCachingPercentage >= this.cacheConf.getCacheOnCompactionThreshold())  {
+      this.numBlocksCachedPerCompaction++;
+      HFile.blocksCachedDuringCompaction.incrementAndGet();
       return true;
     }
     return false;

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1470790&r1=1470789&r2=1470790&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Tue Apr 23 02:49:31 2013
@@ -965,6 +965,10 @@ public class Store extends SchemaConfigu
           + ", size=" + (sf == null? "none" : StringUtils.humanReadableInt(sf.getReader().length()))
           + "; total size for store is "
           + StringUtils.humanReadableInt(storeSize));
+      if (writer != null) {
+        LOG.info(", number of blocks precached="
+            + writer.getHFileWriter().getNumBlocksCachedPerCompaction());
+      }
     } catch (IOException ioe) {
       // rather than leak the status, we abort here, then rethrow the exception
       status.abort(StringUtils.stringifyException(ioe));

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java?rev=1470790&r1=1470789&r2=1470790&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java Tue Apr 23 02:49:31 2013
@@ -176,6 +176,12 @@ public class RegionServerMetrics impleme
       new MetricsTimeVaryingRate("fsReadLatency", registry);
 
   /**
+   * number of blocks cached during compaction
+   */
+  public final MetricsIntValue blocksCachedDuringCompaction =
+      new MetricsIntValue("blocksCachedDuringCompaction", registry);
+
+  /**
    * filesystem p99 read latency outlier for positional read operations
    */
   public final PercentileMetric fsReadLatencyP99 =
@@ -424,6 +430,9 @@ public class RegionServerMetrics impleme
         this.rowLockTime.inc(writeOps, HRegion.getRowLockTime());
       }
 
+      this.blocksCachedDuringCompaction.set(
+          HFile.getBlocksCachedDuringCompactionAndReset());
+
       // push the result
       this.fsReadLatencyP99.pushMetric(this.metricsRecord);
       this.fsCompactionReadLatencyP99.pushMetric(this.metricsRecord);
@@ -454,6 +463,7 @@ public class RegionServerMetrics impleme
       this.quorumReadWins.pushMetric(this.metricsRecord);
       this.quorumReadsExecutedInCurThread.pushMetric(this.metricsRecord);
 
+      this.blocksCachedDuringCompaction.pushMetric(this.metricsRecord);
     }
     this.metricsRecord.update();
   }