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.Random;
24 import java.util.concurrent.BlockingQueue;
25 import java.util.concurrent.atomic.AtomicInteger;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.Abortable;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.classification.InterfaceStability;
34 import org.apache.hadoop.util.StringUtils;
35
36 import com.google.common.base.Preconditions;
37 import com.google.common.base.Strings;
38
39 @InterfaceAudience.Private
40 @InterfaceStability.Evolving
41 public abstract class RpcExecutor {
42 private static final Log LOG = LogFactory.getLog(RpcExecutor.class);
43
44 private final AtomicInteger activeHandlerCount = new AtomicInteger(0);
45 private final List<Thread> handlers;
46 private final int handlerCount;
47 private final String name;
48 private final AtomicInteger failedHandlerCount = new AtomicInteger(0);
49
50 private boolean running;
51
52 private Configuration conf = null;
53 private Abortable abortable = null;
54
55 public RpcExecutor(final String name, final int handlerCount) {
56 this.handlers = new ArrayList<Thread>(handlerCount);
57 this.handlerCount = handlerCount;
58 this.name = Strings.nullToEmpty(name);
59 }
60
61 public RpcExecutor(final String name, final int handlerCount, final Configuration conf,
62 final Abortable abortable) {
63 this(name, handlerCount);
64 this.conf = conf;
65 this.abortable = abortable;
66 }
67
68 public void start(final int port) {
69 running = true;
70 startHandlers(port);
71 }
72
73 public void stop() {
74 running = false;
75 for (Thread handler : handlers) {
76 handler.interrupt();
77 }
78 }
79
80 public int getActiveHandlerCount() {
81 return activeHandlerCount.get();
82 }
83
84
85 public abstract int getQueueLength();
86
87
88 public abstract void dispatch(final CallRunner callTask) throws InterruptedException;
89
90
91 protected abstract List<BlockingQueue<CallRunner>> getQueues();
92
93 protected void startHandlers(final int port) {
94 List<BlockingQueue<CallRunner>> callQueues = getQueues();
95 startHandlers(null, handlerCount, callQueues, 0, callQueues.size(), port);
96 }
97
98 protected void startHandlers(final String nameSuffix, final int numHandlers,
99 final List<BlockingQueue<CallRunner>> callQueues,
100 final int qindex, final int qsize, final int port) {
101 final String threadPrefix = name + Strings.nullToEmpty(nameSuffix);
102 for (int i = 0; i < numHandlers; i++) {
103 final int index = qindex + (i % qsize);
104 Thread t = new Thread(new Runnable() {
105 @Override
106 public void run() {
107 consumerLoop(callQueues.get(index));
108 }
109 });
110 t.setDaemon(true);
111 t.setName(threadPrefix + "RpcServer.handler=" + handlers.size() +
112 ",queue=" + index + ",port=" + port);
113 t.start();
114 LOG.debug(threadPrefix + " Start Handler index=" + handlers.size() + " queue=" + index);
115 handlers.add(t);
116 }
117 }
118
119 protected void consumerLoop(final BlockingQueue<CallRunner> myQueue) {
120 boolean interrupted = false;
121 double handlerFailureThreshhold =
122 conf == null ? 1.0 : conf.getFloat(HConstants.REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT,
123 HConstants.DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT);
124 try {
125 while (running) {
126 try {
127 CallRunner task = myQueue.take();
128 try {
129 activeHandlerCount.incrementAndGet();
130 task.run();
131 } catch (Error e) {
132 int failedCount = failedHandlerCount.incrementAndGet();
133 if (handlerFailureThreshhold >= 0
134 && failedCount > handlerCount * handlerFailureThreshhold) {
135 String message =
136 "Number of failed RpcServer handler exceeded threshhold "
137 + handlerFailureThreshhold + " with failed reason: "
138 + StringUtils.stringifyException(e);
139 if (abortable != null) {
140 abortable.abort(message, e);
141 } else {
142 LOG.error("Received " + StringUtils.stringifyException(e)
143 + " but not aborting due to abortable being null");
144 throw e;
145 }
146 } else {
147 LOG.warn("RpcServer handler threads encountered errors "
148 + StringUtils.stringifyException(e));
149 }
150 } finally {
151 activeHandlerCount.decrementAndGet();
152 }
153 } catch (InterruptedException e) {
154 interrupted = true;
155 }
156 }
157 } finally {
158 if (interrupted) {
159 Thread.currentThread().interrupt();
160 }
161 }
162 }
163
164 public static abstract class QueueBalancer {
165
166
167
168 public abstract int getNextQueue();
169 }
170
171 public static QueueBalancer getBalancer(int queueSize) {
172 Preconditions.checkArgument(queueSize > 0, "Queue size is <= 0, must be at least 1");
173 if (queueSize == 1) {
174 return ONE_QUEUE;
175 } else {
176 return new RandomQueueBalancer(queueSize);
177 }
178 }
179
180
181
182
183 private static QueueBalancer ONE_QUEUE = new QueueBalancer() {
184
185 @Override
186 public int getNextQueue() {
187 return 0;
188 }
189 };
190
191
192
193
194 private static class RandomQueueBalancer extends QueueBalancer {
195 private final int queueSize;
196
197 private final ThreadLocal<Random> threadRandom =
198 new ThreadLocal<Random>() {
199 @Override
200 protected Random initialValue() {
201 return new Random();
202 }
203 };
204
205 public RandomQueueBalancer(int queueSize) {
206 this.queueSize = queueSize;
207 }
208
209 public int getNextQueue() {
210 return threadRandom.get().nextInt(queueSize);
211 }
212 }
213 }