hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From la...@apache.org
Subject svn commit: r1305721 - in /hbase/branches/0.94: CHANGES.txt src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java
Date Tue, 27 Mar 2012 03:33:22 GMT
Author: larsh
Date: Tue Mar 27 03:33:21 2012
New Revision: 1305721

URL: http://svn.apache.org/viewvc?rev=1305721&view=rev
Log:
HBASE-5641 decayingSampleTick1 prevents HBase from shutting down.

Modified:
    hbase/branches/0.94/CHANGES.txt
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java

Modified: hbase/branches/0.94/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/CHANGES.txt?rev=1305721&r1=1305720&r2=1305721&view=diff
==============================================================================
--- hbase/branches/0.94/CHANGES.txt (original)
+++ hbase/branches/0.94/CHANGES.txt Tue Mar 27 03:33:21 2012
@@ -33,6 +33,7 @@ Sub-task
     [HBASE-5497] - Add protobuf as M/R dependency jar (mapred)
     [HBASE-5523] - Fix Delete Timerange logic for KEEP_DELETED_CELLS
     [HBASE-5541] - Avoid holding the rowlock during HLog sync in HRegion.mutateRowWithLocks
+    [HBASE-5641] - decayingSampleTick1 prevents HBase from shutting down.
 
 Bug
 

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java?rev=1305721&r1=1305720&r2=1305721&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java
(original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java
Tue Mar 27 03:33:21 2012
@@ -22,12 +22,12 @@ import java.util.Random;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import org.apache.hadoop.hbase.util.Threads;
-
 /**
  * An exponentially-decaying random sample of {@code long}s. 
  * Uses Cormode et al's forward-decaying priority reservoir sampling method 
@@ -44,7 +44,7 @@ public class ExponentiallyDecayingSample
 
   private static final ScheduledExecutorService TICK_SERVICE = 
       Executors.newScheduledThreadPool(1, 
-          Threads.getNamedThreadFactory("decayingSampleTick"));
+          getNamedDaemonThreadFactory("decayingSampleTick"));
 
   private static volatile long CURRENT_TICK = 
       TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis());
@@ -209,4 +209,18 @@ public class ExponentiallyDecayingSample
   private void unlockForRegularUsage() {
     lock.readLock().unlock();
   }
+
+  private static ThreadFactory getNamedDaemonThreadFactory(final String prefix) {
+    return new ThreadFactory() {
+
+      private final AtomicInteger threadNumber = new AtomicInteger(1);
+      
+      @Override
+      public Thread newThread(Runnable r) {
+        Thread t= new Thread(r, prefix + threadNumber.getAndIncrement());
+        t.setDaemon(true);
+        return t;
+      }
+    };
+  }
 }



Mime
View raw message