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:17:48 UTC

svn commit: r1181536 - /hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java

Author: nspiegelberg
Date: Tue Oct 11 02:17:48 2011
New Revision: 1181536

URL: http://svn.apache.org/viewvc?rev=1181536&view=rev
Log:
NPE Fix for HBASE-1476 with compaction throttling disabled

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=1181536&r1=1181535&r2=1181536&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Tue Oct 11 02:17:48 2011
@@ -74,11 +74,14 @@ public class CompactSplitThread {
         60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
     this.largeCompactions
         .setRejectedExecutionHandler(new CompactionRequest.Rejection());
-    this.smallCompactions = (smallThreads <= 0) ? null
-        : new ThreadPoolExecutor(smallThreads, smallThreads, 60,
-          TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
-    this.smallCompactions
-        .setRejectedExecutionHandler(new CompactionRequest.Rejection());
+    if (smallThreads <= 0) {
+      this.smallCompactions = null;
+    } else {
+      this.smallCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
+          60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
+      this.smallCompactions
+          .setRejectedExecutionHandler(new CompactionRequest.Rejection());
+    }
     this.splits = (ThreadPoolExecutor) Executors
         .newFixedThreadPool(splitThreads);
   }