You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:18:01 UTC

svn commit: r1181539 - in /hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver: HRegion.java compactions/CompactionRequest.java

Author: nspiegelberg
Date: Tue Oct 11 02:18:01 2011
New Revision: 1181539

URL: http://svn.apache.org/viewvc?rev=1181539&view=rev
Log:
Fixed compaction metric reporting after HBASE-3797

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1181539&r1=1181538&r2=1181539&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Tue Oct 11 02:18:01 2011
@@ -950,7 +950,8 @@ public class HRegion implements HeapSize
             return false;
           }
         }
-        LOG.info("Starting compaction on region " + this);
+        LOG.info("Starting compaction on " + cr.getStore() + " in region "
+            + this);
         doRegionCompactionPrep();
         boolean completed = false;
         try {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java?rev=1181539&r1=1181538&r2=1181539&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java Tue Oct 11 02:18:01 2011
@@ -19,7 +19,8 @@ import org.apache.hadoop.util.StringUtil
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Collections2;
 
 /**
  * This class holds all details necessary to run a compaction.
@@ -122,9 +123,16 @@ public class CompactionRequest implement
 
   @Override
     public String toString() {
-      String fsList = Joiner.on(", ").join(Lists.transform(files,
+      String fsList = Joiner.on(", ").join(
+        Collections2.transform(Collections2.filter(files,
+            new Predicate<StoreFile>() {
+              public boolean apply(StoreFile sf) {
+                return sf.getReader() != null;
+              }
+            }),
         new Function<StoreFile, String>() {
           public String apply(StoreFile sf) {
+
             return StringUtils.humanReadableInt(sf.getReader().length());
           }
         }));
@@ -133,7 +141,7 @@ public class CompactionRequest implement
         ", storeName=" + new String(s.getFamily().getName()) +
         ", fileCount=" + files.size() +
         ", fileSize=" + StringUtils.humanReadableInt(totalSize) +
-          " (" + fsList + ")" +
+          ((fsList.isEmpty()) ? "" : " (" + fsList + ")") +
         ", priority=" + p + ", date=" + date;
     }
 
@@ -144,13 +152,13 @@ public class CompactionRequest implement
         return;
       }
       try {
-        long startTime = EnvironmentEdgeManager.currentTimeMillis();
+        long start = EnvironmentEdgeManager.currentTimeMillis();
         boolean completed = r.compact(this);
         long now = EnvironmentEdgeManager.currentTimeMillis();
-        LOG.info(((completed) ? "completed" : "aborted") + " compaction: " + this
-            + ", duration=" + StringUtils.formatTimeDiff(now, startTime));
+        LOG.debug(((completed) ? "completed" : "aborted") + " compaction: " +
+            this + "; duration=" + StringUtils.formatTimeDiff(now, start));
         if (completed) {
-          server.getMetrics().addCompaction(now - startTime, this.totalSize);
+          server.getMetrics().addCompaction(now - start, this.totalSize);
           // degenerate case: blocked regions require recursive enqueues
           if (s.getCompactPriority() <= 0) {
             server.compactSplitThread