HBASE-23364 HRegionServer sometimes does not shut down.
This commit is contained in:
parent
6d251efa04
commit
9c82a65b29
|
@ -34,6 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* LossyCounting utility, bounded data structure that maintains approximate high frequency
|
||||
|
@ -75,7 +76,8 @@ public class LossyCounting<T> {
|
|||
this.data = new ConcurrentHashMap<>();
|
||||
this.listener = listener;
|
||||
calculateCurrentTerm();
|
||||
executor = Executors.newSingleThreadExecutor();
|
||||
executor = Executors.newSingleThreadExecutor(
|
||||
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("lossy-count-%d").build());
|
||||
}
|
||||
|
||||
public LossyCounting(String name, LossyCountingListener listener) {
|
||||
|
|
Loading…
Reference in New Issue