1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.ipc;
19 import static org.junit.Assert.assertEquals;
20 import static org.junit.Assert.assertFalse;
21 import static org.junit.Assert.assertNotEquals;
22 import static org.junit.Assert.assertTrue;
23 import static org.mockito.Matchers.any;
24 import static org.mockito.Matchers.anyObject;
25 import static org.mockito.Matchers.eq;
26 import static org.mockito.Mockito.doAnswer;
27 import static org.mockito.Mockito.mock;
28 import static org.mockito.Mockito.timeout;
29 import static org.mockito.Mockito.verify;
30 import static org.mockito.Mockito.when;
31
32 import java.io.IOException;
33 import java.net.InetSocketAddress;
34 import java.util.ArrayList;
35 import java.util.HashSet;
36 import java.util.List;
37 import java.util.Map;
38 import java.util.Set;
39 import java.util.concurrent.BlockingQueue;
40 import java.util.concurrent.CountDownLatch;
41 import java.util.concurrent.LinkedBlockingQueue;
42 import java.util.concurrent.atomic.AtomicInteger;
43
44 import org.apache.commons.logging.Log;
45 import org.apache.commons.logging.LogFactory;
46 import org.apache.hadoop.conf.Configuration;
47 import org.apache.hadoop.hbase.Abortable;
48 import org.apache.hadoop.hbase.HBaseConfiguration;
49 import org.apache.hadoop.hbase.HConstants;
50 import org.apache.hadoop.hbase.client.Put;
51 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
52 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl;
53 import org.apache.hadoop.hbase.protobuf.RequestConverter;
54 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
55 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
56 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
57 import org.apache.hadoop.hbase.security.User;
58 import org.apache.hadoop.hbase.testclassification.SmallTests;
59 import org.apache.hadoop.hbase.util.Bytes;
60 import org.apache.hadoop.hbase.util.EnvironmentEdge;
61 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
62 import org.apache.hadoop.hbase.util.Threads;
63 import org.junit.Before;
64 import org.junit.Rule;
65 import org.junit.Test;
66 import org.junit.experimental.categories.Category;
67 import org.junit.rules.TestName;
68 import org.mockito.Mockito;
69 import org.mockito.invocation.InvocationOnMock;
70 import org.mockito.stubbing.Answer;
71
72 import com.google.common.collect.ImmutableList;
73 import com.google.common.collect.ImmutableMap;
74 import com.google.common.collect.ImmutableSet;
75 import com.google.common.collect.Maps;
76 import com.google.protobuf.Message;
77
78 @Category(SmallTests.class)
79 public class TestSimpleRpcScheduler {
80 private static final Log LOG = LogFactory.getLog(TestSimpleRpcScheduler.class);
81
82 @Rule
83 public TestName testName = new TestName();
84
85 private final RpcScheduler.Context CONTEXT = new RpcScheduler.Context() {
86 @Override
87 public InetSocketAddress getListenerAddress() {
88 return InetSocketAddress.createUnresolved("127.0.0.1", 1000);
89 }
90 };
91 private Configuration conf;
92
93 @Before
94 public void setUp() {
95 conf = HBaseConfiguration.create();
96 }
97
98 @Test
99 public void testBasic() throws IOException, InterruptedException {
100 PriorityFunction qosFunction = mock(PriorityFunction.class);
101 RpcScheduler scheduler = new SimpleRpcScheduler(
102 conf, 10, 0, 0, qosFunction, 0);
103 scheduler.init(CONTEXT);
104 scheduler.start();
105 CallRunner task = createMockTask();
106 task.setStatus(new MonitoredRPCHandlerImpl());
107 scheduler.dispatch(task);
108 verify(task, timeout(1000)).run();
109 scheduler.stop();
110 }
111
112 @Test
113 public void testHandlerIsolation() throws IOException, InterruptedException {
114 CallRunner generalTask = createMockTask();
115 CallRunner priorityTask = createMockTask();
116 CallRunner replicationTask = createMockTask();
117 List<CallRunner> tasks = ImmutableList.of(
118 generalTask,
119 priorityTask,
120 replicationTask);
121 Map<CallRunner, Integer> qos = ImmutableMap.of(
122 generalTask, 0,
123 priorityTask, HConstants.HIGH_QOS + 1,
124 replicationTask, HConstants.REPLICATION_QOS);
125 PriorityFunction qosFunction = mock(PriorityFunction.class);
126 final Map<CallRunner, Thread> handlerThreads = Maps.newHashMap();
127 final CountDownLatch countDownLatch = new CountDownLatch(tasks.size());
128 Answer<Void> answerToRun = new Answer<Void>() {
129 @Override
130 public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
131 synchronized (handlerThreads) {
132 handlerThreads.put(
133 (CallRunner) invocationOnMock.getMock(),
134 Thread.currentThread());
135 }
136 countDownLatch.countDown();
137 return null;
138 }
139 };
140 for (CallRunner task : tasks) {
141 task.setStatus(new MonitoredRPCHandlerImpl());
142 doAnswer(answerToRun).when(task).run();
143 }
144
145 RpcScheduler scheduler = new SimpleRpcScheduler(
146 conf, 1, 1 ,1, qosFunction, HConstants.HIGH_QOS);
147 scheduler.init(CONTEXT);
148 scheduler.start();
149 for (CallRunner task : tasks) {
150 when(qosFunction.getPriority((RPCProtos.RequestHeader) anyObject(),
151 (Message) anyObject(), (User) anyObject()))
152 .thenReturn(qos.get(task));
153 scheduler.dispatch(task);
154 }
155 for (CallRunner task : tasks) {
156 verify(task, timeout(1000)).run();
157 }
158 scheduler.stop();
159
160
161 countDownLatch.await();
162 assertEquals(3, ImmutableSet.copyOf(handlerThreads.values()).size());
163 }
164
165 private CallRunner createMockTask() {
166 Call call = mock(Call.class);
167 CallRunner task = mock(CallRunner.class);
168 when(task.getCall()).thenReturn(call);
169 return task;
170 }
171
172 @Test
173 public void testRpcScheduler() throws Exception {
174 testRpcScheduler(RpcExecutor.CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
175 testRpcScheduler(RpcExecutor.CALL_QUEUE_TYPE_FIFO_CONF_VALUE);
176 }
177
178 private void testRpcScheduler(final String queueType) throws Exception {
179 Configuration schedConf = HBaseConfiguration.create();
180 schedConf.set(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY, queueType);
181
182 PriorityFunction priority = mock(PriorityFunction.class);
183 when(priority.getPriority(any(RequestHeader.class),
184 any(Message.class), any(User.class)))
185 .thenReturn(HConstants.NORMAL_QOS);
186
187 RpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 1, 1, 1, priority,
188 HConstants.QOS_THRESHOLD);
189 try {
190 scheduler.start();
191
192 CallRunner smallCallTask = mock(CallRunner.class);
193 RpcServer.Call smallCall = mock(RpcServer.Call.class);
194 RequestHeader smallHead = RequestHeader.newBuilder().setCallId(1).build();
195 when(smallCallTask.getCall()).thenReturn(smallCall);
196 when(smallCall.getHeader()).thenReturn(smallHead);
197
198 CallRunner largeCallTask = mock(CallRunner.class);
199 RpcServer.Call largeCall = mock(RpcServer.Call.class);
200 RequestHeader largeHead = RequestHeader.newBuilder().setCallId(50).build();
201 when(largeCallTask.getCall()).thenReturn(largeCall);
202 when(largeCall.getHeader()).thenReturn(largeHead);
203
204 CallRunner hugeCallTask = mock(CallRunner.class);
205 RpcServer.Call hugeCall = mock(RpcServer.Call.class);
206 RequestHeader hugeHead = RequestHeader.newBuilder().setCallId(100).build();
207 when(hugeCallTask.getCall()).thenReturn(hugeCall);
208 when(hugeCall.getHeader()).thenReturn(hugeHead);
209
210 when(priority.getDeadline(eq(smallHead), any(Message.class))).thenReturn(0L);
211 when(priority.getDeadline(eq(largeHead), any(Message.class))).thenReturn(50L);
212 when(priority.getDeadline(eq(hugeHead), any(Message.class))).thenReturn(100L);
213
214 final ArrayList<Integer> work = new ArrayList<Integer>();
215 doAnswerTaskExecution(smallCallTask, work, 10, 250);
216 doAnswerTaskExecution(largeCallTask, work, 50, 250);
217 doAnswerTaskExecution(hugeCallTask, work, 100, 250);
218
219 scheduler.dispatch(smallCallTask);
220 scheduler.dispatch(smallCallTask);
221 scheduler.dispatch(smallCallTask);
222 scheduler.dispatch(hugeCallTask);
223 scheduler.dispatch(smallCallTask);
224 scheduler.dispatch(largeCallTask);
225 scheduler.dispatch(smallCallTask);
226 scheduler.dispatch(smallCallTask);
227
228 while (work.size() < 8) {
229 Thread.sleep(100);
230 }
231
232 int seqSum = 0;
233 int totalTime = 0;
234 for (int i = 0; i < work.size(); ++i) {
235 LOG.debug("Request i=" + i + " value=" + work.get(i));
236 seqSum += work.get(i);
237 totalTime += seqSum;
238 }
239 LOG.debug("Total Time: " + totalTime);
240
241
242
243
244 if (queueType.equals(RpcExecutor.CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
245 assertEquals(530, totalTime);
246 } else if (queueType.equals(RpcExecutor.CALL_QUEUE_TYPE_FIFO_CONF_VALUE)) {
247 assertEquals(930, totalTime);
248 }
249 } finally {
250 scheduler.stop();
251 }
252 }
253
254 @Test
255 public void testScanQueueWithZeroScanRatio() throws Exception {
256 Configuration schedConf = HBaseConfiguration.create();
257 schedConf.setFloat(RpcExecutor.CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 1.0f);
258 schedConf.setFloat(RWQueueRpcExecutor.CALL_QUEUE_READ_SHARE_CONF_KEY, 0.5f);
259 schedConf.setFloat(RWQueueRpcExecutor.CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0f);
260
261 PriorityFunction priority = mock(PriorityFunction.class);
262 when(priority.getPriority(any(RequestHeader.class), any(Message.class),
263 any(User.class))).thenReturn(HConstants.NORMAL_QOS);
264
265 RpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 2, 1, 1, priority,
266 HConstants.QOS_THRESHOLD);
267 assertNotEquals(scheduler, null);
268 }
269
270 @Test
271 public void testScanQueues() throws Exception {
272 Configuration schedConf = HBaseConfiguration.create();
273 schedConf.setFloat(RpcExecutor.CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 1.0f);
274 schedConf.setFloat(RWQueueRpcExecutor.CALL_QUEUE_READ_SHARE_CONF_KEY, 0.7f);
275 schedConf.setFloat(RWQueueRpcExecutor.CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0.5f);
276
277 PriorityFunction priority = mock(PriorityFunction.class);
278 when(priority.getPriority(any(RequestHeader.class), any(Message.class),
279 any(User.class))).thenReturn(HConstants.NORMAL_QOS);
280
281 RpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 3, 1, 1, priority,
282 HConstants.QOS_THRESHOLD);
283 try {
284 scheduler.start();
285
286 CallRunner putCallTask = mock(CallRunner.class);
287 RpcServer.Call putCall = mock(RpcServer.Call.class);
288 putCall.param = RequestConverter.buildMutateRequest(
289 Bytes.toBytes("abc"), new Put(Bytes.toBytes("row")));
290 RequestHeader putHead = RequestHeader.newBuilder().setMethodName("mutate").build();
291 when(putCallTask.getCall()).thenReturn(putCall);
292 when(putCall.getHeader()).thenReturn(putHead);
293
294 CallRunner getCallTask = mock(CallRunner.class);
295 RpcServer.Call getCall = mock(RpcServer.Call.class);
296 RequestHeader getHead = RequestHeader.newBuilder().setMethodName("get").build();
297 when(getCallTask.getCall()).thenReturn(getCall);
298 when(getCall.getHeader()).thenReturn(getHead);
299
300 CallRunner scanCallTask = mock(CallRunner.class);
301 RpcServer.Call scanCall = mock(RpcServer.Call.class);
302 scanCall.param = ScanRequest.newBuilder().build();
303 RequestHeader scanHead = RequestHeader.newBuilder().setMethodName("scan").build();
304 when(scanCallTask.getCall()).thenReturn(scanCall);
305 when(scanCall.getHeader()).thenReturn(scanHead);
306
307 ArrayList<Integer> work = new ArrayList<Integer>();
308 doAnswerTaskExecution(putCallTask, work, 1, 1000);
309 doAnswerTaskExecution(getCallTask, work, 2, 1000);
310 doAnswerTaskExecution(scanCallTask, work, 3, 1000);
311
312
313
314 scheduler.dispatch(putCallTask);
315 scheduler.dispatch(putCallTask);
316 scheduler.dispatch(putCallTask);
317 scheduler.dispatch(getCallTask);
318 scheduler.dispatch(getCallTask);
319 scheduler.dispatch(getCallTask);
320 scheduler.dispatch(scanCallTask);
321 scheduler.dispatch(scanCallTask);
322 scheduler.dispatch(scanCallTask);
323
324 while (work.size() < 6) {
325 Thread.sleep(100);
326 }
327
328 for (int i = 0; i < work.size() - 2; i += 3) {
329 assertNotEquals(work.get(i + 0), work.get(i + 1));
330 assertNotEquals(work.get(i + 0), work.get(i + 2));
331 assertNotEquals(work.get(i + 1), work.get(i + 2));
332 }
333 } finally {
334 scheduler.stop();
335 }
336 }
337
338 private void doAnswerTaskExecution(final CallRunner callTask,
339 final ArrayList<Integer> results, final int value, final int sleepInterval) {
340 callTask.setStatus(new MonitoredRPCHandlerImpl());
341 doAnswer(new Answer<Object>() {
342 @Override
343 public Object answer(InvocationOnMock invocation) {
344 synchronized (results) {
345 results.add(value);
346 }
347 Threads.sleepWithoutInterrupt(sleepInterval);
348 return null;
349 }
350 }).when(callTask).run();
351 }
352
353 private static void waitUntilQueueEmpty(SimpleRpcScheduler scheduler)
354 throws InterruptedException {
355 while (scheduler.getGeneralQueueLength() > 0) {
356 Thread.sleep(100);
357 }
358 }
359
360 @Test
361 public void testSoftAndHardQueueLimits() throws Exception {
362 Configuration schedConf = HBaseConfiguration.create();
363
364 schedConf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 0);
365 schedConf.setInt("hbase.ipc.server.max.callqueue.length", 5);
366
367 PriorityFunction priority = mock(PriorityFunction.class);
368 when(priority.getPriority(any(RequestHeader.class), any(Message.class),
369 any(User.class))).thenReturn(HConstants.NORMAL_QOS);
370 SimpleRpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 0, 0, 0, priority,
371 HConstants.QOS_THRESHOLD);
372 try {
373 scheduler.start();
374
375 CallRunner putCallTask = mock(CallRunner.class);
376 RpcServer.Call putCall = mock(RpcServer.Call.class);
377 putCall.param = RequestConverter.buildMutateRequest(
378 Bytes.toBytes("abc"), new Put(Bytes.toBytes("row")));
379 RequestHeader putHead = RequestHeader.newBuilder().setMethodName("mutate").build();
380 when(putCallTask.getCall()).thenReturn(putCall);
381 when(putCall.getHeader()).thenReturn(putHead);
382
383 assertTrue(scheduler.dispatch(putCallTask));
384 waitUntilQueueEmpty(scheduler);
385
386 schedConf.setInt("hbase.ipc.server.max.callqueue.length", 0);
387 scheduler.onConfigurationChange(schedConf);
388 assertFalse(scheduler.dispatch(putCallTask));
389 waitUntilQueueEmpty(scheduler);
390 schedConf.setInt("hbase.ipc.server.max.callqueue.length", 1);
391 scheduler.onConfigurationChange(schedConf);
392 assertTrue(scheduler.dispatch(putCallTask));
393 } finally {
394 scheduler.stop();
395 }
396 }
397
398 private static final class CoDelEnvironmentEdge implements EnvironmentEdge {
399
400 private final BlockingQueue<Long> timeQ = new LinkedBlockingQueue<>();
401
402 private long offset;
403
404 private final Set<String> threadNamePrefixs = new HashSet<>();
405
406 @Override
407 public long currentTime() {
408 for (String threadNamePrefix : threadNamePrefixs) {
409 String threadName = Thread.currentThread().getName();
410 if (threadName.startsWith(threadNamePrefix)) {
411 return timeQ.poll().longValue() + offset;
412 }
413 }
414 return System.currentTimeMillis();
415 }
416 }
417
418 @Test
419 public void testCoDelScheduling() throws Exception {
420 CoDelEnvironmentEdge envEdge = new CoDelEnvironmentEdge();
421 envEdge.threadNamePrefixs.add("RpcServer.default.FPBQ.Codel.handler");
422 Configuration schedConf = HBaseConfiguration.create();
423 schedConf.setInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, 250);
424
425 schedConf.set(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY,
426 RpcExecutor.CALL_QUEUE_TYPE_CODEL_CONF_VALUE);
427
428 PriorityFunction priority = mock(PriorityFunction.class);
429 when(priority.getPriority(any(RPCProtos.RequestHeader.class), any(Message.class),
430 any(User.class))).thenReturn(HConstants.NORMAL_QOS);
431 SimpleRpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 1, 1, 1, priority,
432 HConstants.QOS_THRESHOLD);
433 try {
434 scheduler.start();
435 EnvironmentEdgeManager.injectEdge(envEdge);
436 envEdge.offset = 5;
437
438 for (int i = 0; i < 100; i++) {
439 long time = System.currentTimeMillis();
440 envEdge.timeQ.put(time);
441 CallRunner cr = getMockedCallRunner(time, 2);
442 scheduler.dispatch(cr);
443 }
444
445 waitUntilQueueEmpty(scheduler);
446 Thread.sleep(100);
447 assertEquals("None of these calls should have been discarded", 0,
448 scheduler.getNumGeneralCallsDropped());
449
450 envEdge.offset = 151;
451
452 for (int i = 0; i < 20; i++) {
453 long time = System.currentTimeMillis();
454 envEdge.timeQ.put(time);
455 CallRunner cr = getMockedCallRunner(time, 2);
456 scheduler.dispatch(cr);
457 }
458
459
460 waitUntilQueueEmpty(scheduler);
461 Thread.sleep(100);
462 assertEquals("None of these calls should have been discarded", 0,
463 scheduler.getNumGeneralCallsDropped());
464
465 envEdge.offset = 2000;
466
467 for (int i = 0; i < 60; i++) {
468 long time = System.currentTimeMillis();
469 envEdge.timeQ.put(time);
470 CallRunner cr = getMockedCallRunner(time, 100);
471 scheduler.dispatch(cr);
472 }
473
474
475 waitUntilQueueEmpty(scheduler);
476 Thread.sleep(100);
477 assertTrue(
478 "There should have been at least 12 calls dropped however there were "
479 + scheduler.getNumGeneralCallsDropped(),
480 scheduler.getNumGeneralCallsDropped() > 12);
481 } finally {
482 scheduler.stop();
483 }
484 }
485
486 @Test
487 public void testFastPathBalancedQueueRpcExecutorWithQueueLength0() throws Exception {
488 String name = testName.getMethodName();
489 int handlerCount = 1;
490 String callQueueType = RpcExecutor.CALL_QUEUE_TYPE_CODEL_CONF_VALUE;
491 int maxQueueLength = 0;
492 PriorityFunction priority = mock(PriorityFunction.class);
493 Configuration conf = HBaseConfiguration.create();
494 Abortable abortable = mock(Abortable.class);
495 FastPathBalancedQueueRpcExecutor executor =
496 Mockito.spy(new FastPathBalancedQueueRpcExecutor(name,
497 handlerCount, callQueueType, maxQueueLength, priority, conf, abortable));
498 CallRunner task = mock(CallRunner.class);
499 assertFalse(executor.dispatch(task));
500
501
502 Mockito.verify(executor, Mockito.never()).getHandler(Mockito.anyString(), Mockito.anyDouble(),
503 Mockito.anyInt(), (BlockingQueue<CallRunner>) Mockito.any(), (AtomicInteger) Mockito.any(),
504 (AtomicInteger) Mockito.any(), (Abortable) Mockito.any());
505 }
506
507
508
509 private CallRunner getMockedCallRunner(long timestamp, final long sleepTime) throws IOException {
510 final RpcServer.Call putCall = mock(RpcServer.Call.class);
511
512 putCall.timestamp = timestamp;
513 putCall.param = RequestConverter.buildMutateRequest(
514 Bytes.toBytes("abc"), new Put(Bytes.toBytes("row")));
515
516 RPCProtos.RequestHeader putHead = RPCProtos.RequestHeader.newBuilder()
517 .setMethodName("mutate")
518 .build();
519 when(putCall.getSize()).thenReturn(9L);
520 when(putCall.getHeader()).thenReturn(putHead);
521
522 CallRunner cr = new CallRunner(null, putCall) {
523 public void run() {
524 try {
525 LOG.warn("Sleeping for " + sleepTime);
526 Thread.sleep(sleepTime);
527 LOG.warn("Done Sleeping for " + sleepTime);
528 } catch (InterruptedException e) {
529 }
530 }
531 public Call getCall() {
532 return putCall;
533 }
534
535 public void drop() {}
536 };
537
538 return cr;
539 }
540 }