1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.ipc;
20
21 import java.util.ArrayList;
22 import java.util.List;
23 import java.util.concurrent.BlockingQueue;
24 import java.util.concurrent.atomic.AtomicInteger;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29 import org.apache.hadoop.hbase.classification.InterfaceStability;
30
31 import com.google.common.base.Strings;
32
33 @InterfaceAudience.Private
34 @InterfaceStability.Evolving
35 public abstract class RpcExecutor {
36 private static final Log LOG = LogFactory.getLog(RpcExecutor.class);
37
38 private final AtomicInteger activeHandlerCount = new AtomicInteger(0);
39 private final List<Thread> handlers;
40 private final int handlerCount;
41 private final String name;
42
43 private boolean running;
44
45 public RpcExecutor(final String name, final int handlerCount) {
46 this.handlers = new ArrayList<Thread>(handlerCount);
47 this.handlerCount = handlerCount;
48 this.name = Strings.nullToEmpty(name);
49 }
50
51 public void start(final int port) {
52 running = true;
53 startHandlers(port);
54 }
55
56 public void stop() {
57 running = false;
58 for (Thread handler : handlers) {
59 handler.interrupt();
60 }
61 }
62
63 public int getActiveHandlerCount() {
64 return activeHandlerCount.get();
65 }
66
67
68 public abstract int getQueueLength();
69
70
71 public abstract void dispatch(final CallRunner callTask) throws InterruptedException;
72
73
74 protected abstract List<BlockingQueue<CallRunner>> getQueues();
75
76 protected void startHandlers(final int port) {
77 List<BlockingQueue<CallRunner>> callQueues = getQueues();
78 startHandlers(null, handlerCount, callQueues, 0, callQueues.size(), port);
79 }
80
81 protected void startHandlers(final String nameSuffix, final int numHandlers,
82 final List<BlockingQueue<CallRunner>> callQueues,
83 final int qindex, final int qsize, final int port) {
84 final String threadPrefix = name + Strings.nullToEmpty(nameSuffix);
85 for (int i = 0; i < numHandlers; i++) {
86 final int index = qindex + (i % qsize);
87 Thread t = new Thread(new Runnable() {
88 @Override
89 public void run() {
90 consumerLoop(callQueues.get(index));
91 }
92 });
93 t.setDaemon(true);
94 t.setName(threadPrefix + "RpcServer.handler=" + handlers.size() +
95 ",queue=" + index + ",port=" + port);
96 t.start();
97 LOG.debug(threadPrefix + " Start Handler index=" + handlers.size() + " queue=" + index);
98 handlers.add(t);
99 }
100 }
101
102 protected void consumerLoop(final BlockingQueue<CallRunner> myQueue) {
103 boolean interrupted = false;
104 try {
105 while (running) {
106 try {
107 CallRunner task = myQueue.take();
108 try {
109 activeHandlerCount.incrementAndGet();
110 task.run();
111 } catch (Error e) {
112 LOG.error("RpcServer handler thread throws error: ", e);
113 throw e;
114 } catch (RuntimeException e) {
115 LOG.error("RpcServer handler thread throws exception: ", e);
116 throw e;
117 } finally {
118 activeHandlerCount.decrementAndGet();
119 }
120 } catch (InterruptedException e) {
121 interrupted = true;
122 }
123 }
124 } finally {
125 if (interrupted) {
126 Thread.currentThread().interrupt();
127 }
128 }
129 }
130 }