[
https://issues.apache.org/jira/browse/HBASE-3808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13037487#comment-13037487
]
Brock Noland commented on HBASE-3808:
-------------------------------------
@stack
Here is what I think you are looking for implemented for ServerShutdownHandler
and ExecutorService$Executor. How far am I off track?
{noformat}
Index:
src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
===================================================================
---
src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
(revision 1125781)
+++
src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
(working copy)
@@ -88,6 +88,11 @@
}
@Override
+ public String toString() {
+ return getClass().getSimpleName() + "-" + server.getServerName();
+ }
+
+ @Override
public void process() throws IOException {
final ServerName serverName = this.serverName;
Index: src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
===================================================================
--- src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
(revision 1125781)
+++ src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
(working copy)
@@ -27,6 +27,7 @@
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -188,7 +189,7 @@
List<Runnable> wasRunning =
entry.getValue().threadPoolExecutor.shutdownNow();
if (!wasRunning.isEmpty()) {
- LOG.info(entry.getKey() + " had " + wasRunning + " on shutdown");
+ LOG.info(entry.getValue() + " had " + wasRunning + " on shutdown");
}
}
this.executorMap.clear();
@@ -255,9 +256,12 @@
final BlockingQueue<Runnable> q = new LinkedBlockingQueue<Runnable>();
private final String name;
private final Map<EventHandler.EventType, EventHandlerListener>
eventHandlerListeners;
-
+ private static final AtomicLong seqids = new AtomicLong(0);
+ private final long id;
+
protected Executor(String name, int maxThreads,
final Map<EventHandler.EventType, EventHandlerListener>
eventHandlerListeners) {
+ this.id = seqids.incrementAndGet();
this.name = name;
this.eventHandlerListeners = eventHandlerListeners;
// create the thread pool executor
@@ -283,5 +287,9 @@
}
this.threadPoolExecutor.execute(event);
}
+
+ public String toString() {
+ return getClass().getSimpleName() + "-" + id + "-" + name;
+ }
}
}
{noformat}
> Implement Executor.toString for master handlers at least
> --------------------------------------------------------
>
> Key: HBASE-3808
> URL: https://issues.apache.org/jira/browse/HBASE-3808
> Project: HBase
> Issue Type: Bug
> Reporter: stack
> Priority: Minor
>
> On shutdown, if still outstanding Executors queued then when ExecutorService
> lists what is outstanding, the list will be other than a list of default
> toString implementations of ServerShutdownHandler objects.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira