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.Comparator;
23 import java.util.List;
24 import java.util.Locale;
25 import java.util.concurrent.BlockingQueue;
26 import java.util.concurrent.LinkedBlockingQueue;
27 import java.util.concurrent.ThreadLocalRandom;
28 import java.util.concurrent.atomic.AtomicInteger;
29 import java.util.concurrent.atomic.AtomicLong;
30
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.hbase.Abortable;
35 import org.apache.hadoop.hbase.HConstants;
36 import org.apache.hadoop.hbase.classification.InterfaceAudience;
37 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
38 import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
39 import org.apache.hadoop.hbase.util.ReflectionUtils;
40 import org.apache.hadoop.util.StringUtils;
41
42 import com.google.common.base.Preconditions;
43 import com.google.common.base.Strings;
44
45
46
47
48
49 @InterfaceAudience.Private
50 public abstract class RpcExecutor {
51 private static final Log LOG = LogFactory.getLog(RpcExecutor.class);
52
53 protected static final int DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT = 250;
54 public static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY = "hbase.ipc.server.callqueue.handler.factor";
55
56
57 public static final String QUEUE_MAX_CALL_DELAY_CONF_KEY = "hbase.ipc.server.queue.max.call.delay";
58
59
60
61
62
63
64 public static final String CALL_QUEUE_TYPE_CODEL_CONF_VALUE = "codel";
65 public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
66 public static final String CALL_QUEUE_TYPE_FIFO_CONF_VALUE = "fifo";
67 public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
68 public static final String CALL_QUEUE_TYPE_CONF_DEFAULT = CALL_QUEUE_TYPE_FIFO_CONF_VALUE;
69
70
71 public static final String CALL_QUEUE_CODEL_TARGET_DELAY = "hbase.ipc.server.callqueue.codel.target.delay";
72 public static final String CALL_QUEUE_CODEL_INTERVAL = "hbase.ipc.server.callqueue.codel.interval";
73 public static final String CALL_QUEUE_CODEL_LIFO_THRESHOLD = "hbase.ipc.server.callqueue.codel.lifo.threshold";
74
75 public static final int CALL_QUEUE_CODEL_DEFAULT_TARGET_DELAY = 100;
76 public static final int CALL_QUEUE_CODEL_DEFAULT_INTERVAL = 100;
77 public static final double CALL_QUEUE_CODEL_DEFAULT_LIFO_THRESHOLD = 0.8;
78
79 private AtomicLong numGeneralCallsDropped = new AtomicLong();
80 private AtomicLong numLifoModeSwitches = new AtomicLong();
81
82 protected final int numCallQueues;
83 protected final List<BlockingQueue<CallRunner>> queues;
84 private final Class<? extends BlockingQueue> queueClass;
85 private final Object[] queueInitArgs;
86
87 private final PriorityFunction priority;
88
89 protected volatile int currentQueueLimit;
90
91 private final AtomicInteger activeHandlerCount = new AtomicInteger(0);
92 private final List<Handler> handlers;
93 private final int handlerCount;
94 private final AtomicInteger failedHandlerCount = new AtomicInteger(0);
95
96 private String name;
97 private boolean running;
98
99 private Configuration conf = null;
100 private Abortable abortable = null;
101
102 @Deprecated
103 public RpcExecutor(final String name, final int handlerCount, final int numCallQueues) {
104 this.name = Strings.nullToEmpty(name);
105 this.handlers = new ArrayList<Handler>(handlerCount);
106 this.handlerCount = handlerCount;
107 this.numCallQueues = numCallQueues;
108 this.queues = new ArrayList<>(this.numCallQueues);
109 this.queueClass = null;
110 this.queueInitArgs = new Object[0];
111 this.priority = null;
112 }
113
114 @Deprecated
115 public RpcExecutor(final String name, final int handlerCount, final int numCallQueues,
116 final Configuration conf, final Abortable abortable) {
117 this(name, handlerCount, numCallQueues);
118 this.conf = conf;
119 this.abortable = abortable;
120 }
121
122 public RpcExecutor(final String name, final int handlerCount, final int maxQueueLength,
123 final PriorityFunction priority, final Configuration conf, final Abortable abortable) {
124 this(name, handlerCount, conf.get(CALL_QUEUE_TYPE_CONF_KEY,
125 CALL_QUEUE_TYPE_CONF_DEFAULT), maxQueueLength, priority, conf, abortable);
126 }
127
128 public RpcExecutor(final String name, final int handlerCount, final String callQueueType,
129 final int maxQueueLength, final PriorityFunction priority, final Configuration conf,
130 final Abortable abortable) {
131 this.name = Strings.nullToEmpty(name);
132 this.conf = conf;
133 this.abortable = abortable;
134
135 float callQueuesHandlersFactor = this.conf.getFloat(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0);
136 if (Float.compare(callQueuesHandlersFactor, 1.0f) > 0 ||
137 Float.compare(0.0f, callQueuesHandlersFactor) > 0) {
138 LOG.warn(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY +
139 " is *ILLEGAL*, it should be in range [0.0, 1.0]");
140
141 if (Float.compare(callQueuesHandlersFactor, 1.0f) > 0) {
142 LOG.warn("Set " + CALL_QUEUE_HANDLER_FACTOR_CONF_KEY + " 1.0f");
143 callQueuesHandlersFactor = 1.0f;
144 } else {
145
146
147 LOG.warn("Set " + CALL_QUEUE_HANDLER_FACTOR_CONF_KEY + " default value 0.0f");
148 }
149 }
150 this.numCallQueues = computeNumCallQueues(handlerCount, callQueuesHandlersFactor);
151 this.queues = new ArrayList<>(this.numCallQueues);
152
153 this.handlerCount = Math.max(handlerCount, this.numCallQueues);
154 this.handlers = new ArrayList<>(this.handlerCount);
155
156 this.priority = priority;
157
158 if (isDeadlineQueueType(callQueueType)) {
159 this.name += ".Deadline";
160 this.queueInitArgs = new Object[] { maxQueueLength,
161 new CallPriorityComparator(conf, this.priority) };
162 this.queueClass = BoundedPriorityBlockingQueue.class;
163 } else if (isCodelQueueType(callQueueType)) {
164 this.name += ".Codel";
165 int codelTargetDelay = conf.getInt(CALL_QUEUE_CODEL_TARGET_DELAY,
166 CALL_QUEUE_CODEL_DEFAULT_TARGET_DELAY);
167 int codelInterval = conf.getInt(CALL_QUEUE_CODEL_INTERVAL, CALL_QUEUE_CODEL_DEFAULT_INTERVAL);
168 double codelLifoThreshold = conf.getDouble(CALL_QUEUE_CODEL_LIFO_THRESHOLD,
169 CALL_QUEUE_CODEL_DEFAULT_LIFO_THRESHOLD);
170 queueInitArgs = new Object[] { maxQueueLength, codelTargetDelay, codelInterval,
171 codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches };
172 queueClass = AdaptiveLifoCoDelCallQueue.class;
173 } else {
174 this.name += ".Fifo";
175 queueInitArgs = new Object[] { maxQueueLength };
176 queueClass = LinkedBlockingQueue.class;
177 }
178
179 LOG.info("RpcExecutor " + " name " + " using " + callQueueType
180 + " as call queue; numCallQueues=" + numCallQueues + "; maxQueueLength=" + maxQueueLength
181 + "; handlerCount=" + handlerCount);
182 }
183
184 protected int computeNumCallQueues(final int handlerCount, final float callQueuesHandlersFactor) {
185 return Math.max(1, (int) Math.round(handlerCount * callQueuesHandlersFactor));
186 }
187
188 protected void initializeQueues(final int numQueues) {
189 if (queueInitArgs.length > 0) {
190 currentQueueLimit = (int) queueInitArgs[0];
191 queueInitArgs[0] = Math.max((int) queueInitArgs[0], DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT);
192 }
193 for (int i = 0; i < numQueues; ++i) {
194 queues
195 .add((BlockingQueue<CallRunner>) ReflectionUtils.newInstance(queueClass, queueInitArgs));
196 }
197 }
198
199 public void start(final int port) {
200 running = true;
201 startHandlers(port);
202 }
203
204 public void stop() {
205 running = false;
206 for (Thread handler : handlers) {
207 handler.interrupt();
208 }
209 }
210
211
212 public abstract boolean dispatch(final CallRunner callTask) throws InterruptedException;
213
214
215 public List<BlockingQueue<CallRunner>> getQueues() {
216 return queues;
217 }
218
219 protected void startHandlers(final int port) {
220 List<BlockingQueue<CallRunner>> callQueues = getQueues();
221 startHandlers(null, handlerCount, callQueues, 0, callQueues.size(), port, activeHandlerCount);
222 }
223
224
225
226
227 protected Handler getHandler(final String name, final double handlerFailureThreshhold,
228 final BlockingQueue<CallRunner> q, final AtomicInteger activeHandlerCount) {
229 return new Handler(name, handlerFailureThreshhold, q, activeHandlerCount);
230 }
231
232
233
234
235 protected void startHandlers(final String nameSuffix, final int numHandlers,
236 final List<BlockingQueue<CallRunner>> callQueues, final int qindex, final int qsize,
237 final int port, final AtomicInteger activeHandlerCount) {
238 final String threadPrefix = name + Strings.nullToEmpty(nameSuffix);
239 double handlerFailureThreshhold = conf == null ? 1.0 : conf.getDouble(
240 HConstants.REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT,
241 HConstants.DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT);
242 for (int i = 0; i < numHandlers; i++) {
243 final int index = qindex + (i % qsize);
244 String name = "RpcServer." + threadPrefix + ".handler=" + handlers.size() + ",queue=" + index
245 + ",port=" + port;
246 Handler handler = getHandler(name, handlerFailureThreshhold, callQueues.get(index),
247 activeHandlerCount);
248 handler.start();
249 LOG.debug("Started " + name);
250 handlers.add(handler);
251 }
252 }
253
254
255
256
257 protected class Handler extends Thread {
258
259
260
261 final BlockingQueue<CallRunner> q;
262
263 final double handlerFailureThreshhold;
264
265
266 final AtomicInteger activeHandlerCount;
267
268 Handler(final String name, final double handlerFailureThreshhold,
269 final BlockingQueue<CallRunner> q, final AtomicInteger activeHandlerCount) {
270 super(name);
271 setDaemon(true);
272 this.q = q;
273 this.handlerFailureThreshhold = handlerFailureThreshhold;
274 this.activeHandlerCount = activeHandlerCount;
275 }
276
277
278
279
280
281 protected CallRunner getCallRunner() throws InterruptedException {
282 return this.q.take();
283 }
284
285 @Override
286 public void run() {
287 boolean interrupted = false;
288 try {
289 while (running) {
290 try {
291 run(getCallRunner());
292 } catch (InterruptedException e) {
293 interrupted = true;
294 }
295 }
296 } catch (Exception e) {
297 LOG.warn(e);
298 throw e;
299 } finally {
300 if (interrupted) {
301 Thread.currentThread().interrupt();
302 }
303 }
304 }
305
306 private void run(CallRunner cr) {
307 MonitoredRPCHandler status = RpcServer.getStatus();
308 cr.setStatus(status);
309 try {
310 this.activeHandlerCount.incrementAndGet();
311 cr.run();
312 } catch (Throwable e) {
313 if (e instanceof Error) {
314 int failedCount = failedHandlerCount.incrementAndGet();
315 if (this.handlerFailureThreshhold >= 0
316 && failedCount > handlerCount * this.handlerFailureThreshhold) {
317 String message = "Number of failed RpcServer handler runs exceeded threshhold "
318 + this.handlerFailureThreshhold + "; reason: " + StringUtils.stringifyException(e);
319 if (abortable != null) {
320 abortable.abort(message, e);
321 } else {
322 LOG.error("Error but can't abort because abortable is null: "
323 + StringUtils.stringifyException(e));
324 throw e;
325 }
326 } else {
327 LOG.warn("Handler errors " + StringUtils.stringifyException(e));
328 }
329 } else {
330 LOG.warn("Handler exception " + StringUtils.stringifyException(e));
331 }
332 } finally {
333 this.activeHandlerCount.decrementAndGet();
334 }
335 }
336 }
337
338 public static abstract class QueueBalancer {
339
340
341
342 public abstract int getNextQueue();
343 }
344
345 public static QueueBalancer getBalancer(int queueSize) {
346 Preconditions.checkArgument(queueSize > 0, "Queue size is <= 0, must be at least 1");
347 if (queueSize == 1) {
348 return ONE_QUEUE;
349 } else {
350 return new RandomQueueBalancer(queueSize);
351 }
352 }
353
354
355
356
357 private static QueueBalancer ONE_QUEUE = new QueueBalancer() {
358 @Override
359 public int getNextQueue() {
360 return 0;
361 }
362 };
363
364
365
366
367 private static class RandomQueueBalancer extends QueueBalancer {
368 private final int queueSize;
369
370 public RandomQueueBalancer(int queueSize) {
371 this.queueSize = queueSize;
372 }
373
374 public int getNextQueue() {
375 return ThreadLocalRandom.current().nextInt(queueSize);
376 }
377 }
378
379
380
381
382
383
384
385 private static class CallPriorityComparator implements Comparator<CallRunner> {
386 private final static int DEFAULT_MAX_CALL_DELAY = 5000;
387
388 private final PriorityFunction priority;
389 private final int maxDelay;
390
391 public CallPriorityComparator(final Configuration conf, final PriorityFunction priority) {
392 this.priority = priority;
393 this.maxDelay = conf.getInt(QUEUE_MAX_CALL_DELAY_CONF_KEY, DEFAULT_MAX_CALL_DELAY);
394 }
395
396 @Override
397 public int compare(CallRunner a, CallRunner b) {
398 RpcServer.Call callA = a.getCall();
399 RpcServer.Call callB = b.getCall();
400 long deadlineA = priority.getDeadline(callA.getHeader(), callA.param);
401 long deadlineB = priority.getDeadline(callB.getHeader(), callB.param);
402 deadlineA = callA.timestamp + Math.min(deadlineA, maxDelay);
403 deadlineB = callB.timestamp + Math.min(deadlineB, maxDelay);
404 return Long.compare(deadlineA, deadlineB);
405 }
406 }
407
408 public static boolean isDeadlineQueueType(final String callQueueType) {
409 return callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
410 }
411
412 public static boolean isCodelQueueType(final String callQueueType) {
413 return callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE);
414 }
415
416 public static boolean isFifoQueueType(final String callQueueType) {
417 return callQueueType.equals(CALL_QUEUE_TYPE_FIFO_CONF_VALUE);
418 }
419
420 public long getNumGeneralCallsDropped() {
421 return numGeneralCallsDropped.get();
422 }
423
424 public long getNumLifoModeSwitches() {
425 return numLifoModeSwitches.get();
426 }
427
428 public int getActiveHandlerCount() {
429 return activeHandlerCount.get();
430 }
431
432 public int getActiveWriteHandlerCount() {
433 return 0;
434 }
435
436 public int getActiveReadHandlerCount() {
437 return 0;
438 }
439
440 public int getActiveScanHandlerCount() {
441 return 0;
442 }
443
444
445 public int getQueueLength() {
446 int length = 0;
447 for (final BlockingQueue<CallRunner> queue: queues) {
448 length += queue.size();
449 }
450 return length;
451 }
452
453 public int getReadQueueLength() {
454 return 0;
455 }
456
457 public int getScanQueueLength() {
458 return 0;
459 }
460
461 public int getWriteQueueLength() {
462 return 0;
463 }
464
465 public String getName() {
466 return this.name;
467 }
468
469
470
471
472
473 public void resizeQueues(Configuration conf) {
474 String configKey = RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH;
475 if (name != null && name.toLowerCase(Locale.ROOT).contains("priority")) {
476 configKey = RpcScheduler.IPC_SERVER_PRIORITY_MAX_CALLQUEUE_LENGTH;
477 }
478 currentQueueLimit = conf.getInt(configKey, currentQueueLimit);
479 }
480
481 public void onConfigurationChange(Configuration conf) {
482
483 int codelTargetDelay = conf.getInt(CALL_QUEUE_CODEL_TARGET_DELAY,
484 CALL_QUEUE_CODEL_DEFAULT_TARGET_DELAY);
485 int codelInterval = conf.getInt(CALL_QUEUE_CODEL_INTERVAL, CALL_QUEUE_CODEL_DEFAULT_INTERVAL);
486 double codelLifoThreshold = conf.getDouble(CALL_QUEUE_CODEL_LIFO_THRESHOLD,
487 CALL_QUEUE_CODEL_DEFAULT_LIFO_THRESHOLD);
488
489 for (BlockingQueue<CallRunner> queue : queues) {
490 if (queue instanceof AdaptiveLifoCoDelCallQueue) {
491 ((AdaptiveLifoCoDelCallQueue) queue).updateTunables(codelTargetDelay, codelInterval,
492 codelLifoThreshold);
493 }
494 }
495 }
496 }