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