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
20 import com.google.common.collect.ImmutableList;
21 import com.google.common.collect.ImmutableMap;
22 import com.google.common.collect.ImmutableSet;
23 import com.google.common.collect.Maps;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27
28 import com.google.protobuf.Message;
29
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.HBaseConfiguration;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.client.Put;
34 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl;
35 import org.apache.hadoop.hbase.security.User;
36 import org.apache.hadoop.hbase.testclassification.SmallTests;
37 import org.apache.hadoop.hbase.client.Put;
38 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
39 import org.apache.hadoop.hbase.protobuf.RequestConverter;
40 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
41 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
42 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
43 import org.apache.hadoop.hbase.util.Bytes;
44 import org.apache.hadoop.hbase.util.Threads;
45 import org.junit.Before;
46 import org.junit.Test;
47 import org.junit.experimental.categories.Category;
48 import org.mockito.invocation.InvocationOnMock;
49 import org.mockito.stubbing.Answer;
50
51 import java.io.IOException;
52 import java.net.InetSocketAddress;
53 import java.util.ArrayList;
54 import java.util.List;
55 import java.util.Map;
56 import java.util.concurrent.CountDownLatch;
57
58 import static org.junit.Assert.assertEquals;
59 import static org.junit.Assert.assertNotEquals;
60 import static org.mockito.Matchers.any;
61 import static org.mockito.Matchers.anyObject;
62 import static org.mockito.Matchers.eq;
63 import static org.mockito.Mockito.doAnswer;
64 import static org.mockito.Mockito.mock;
65 import static org.mockito.Mockito.timeout;
66 import static org.mockito.Mockito.verify;
67 import static org.mockito.Mockito.when;
68
69 @Category(SmallTests.class)
70 public class TestSimpleRpcScheduler {
71 public static final Log LOG = LogFactory.getLog(TestSimpleRpcScheduler.class);
72
73 private final RpcScheduler.Context CONTEXT = new RpcScheduler.Context() {
74 @Override
75 public InetSocketAddress getListenerAddress() {
76 return InetSocketAddress.createUnresolved("127.0.0.1", 1000);
77 }
78 };
79 private Configuration conf;
80
81 @Before
82 public void setUp() {
83 conf = HBaseConfiguration.create();
84 }
85
86 @Test
87 public void testBasic() throws IOException, InterruptedException {
88 PriorityFunction qosFunction = mock(PriorityFunction.class);
89 RpcScheduler scheduler = new SimpleRpcScheduler(
90 conf, 10, 0, 0, qosFunction, 0);
91 scheduler.init(CONTEXT);
92 scheduler.start();
93 CallRunner task = createMockTask();
94 task.setStatus(new MonitoredRPCHandlerImpl());
95 scheduler.dispatch(task);
96 verify(task, timeout(1000)).run();
97 scheduler.stop();
98 }
99
100 @Test
101 public void testHandlerIsolation() throws IOException, InterruptedException {
102 CallRunner generalTask = createMockTask();
103 CallRunner priorityTask = createMockTask();
104 CallRunner replicationTask = createMockTask();
105 List<CallRunner> tasks = ImmutableList.of(
106 generalTask,
107 priorityTask,
108 replicationTask);
109 Map<CallRunner, Integer> qos = ImmutableMap.of(
110 generalTask, 0,
111 priorityTask, HConstants.HIGH_QOS + 1,
112 replicationTask, HConstants.REPLICATION_QOS);
113 PriorityFunction qosFunction = mock(PriorityFunction.class);
114 final Map<CallRunner, Thread> handlerThreads = Maps.newHashMap();
115 final CountDownLatch countDownLatch = new CountDownLatch(tasks.size());
116 Answer<Void> answerToRun = new Answer<Void>() {
117 @Override
118 public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
119 synchronized (handlerThreads) {
120 handlerThreads.put(
121 (CallRunner) invocationOnMock.getMock(),
122 Thread.currentThread());
123 }
124 countDownLatch.countDown();
125 return null;
126 }
127 };
128 for (CallRunner task : tasks) {
129 task.setStatus(new MonitoredRPCHandlerImpl());
130 doAnswer(answerToRun).when(task).run();
131 }
132
133 RpcScheduler scheduler = new SimpleRpcScheduler(
134 conf, 1, 1 ,1, qosFunction, HConstants.HIGH_QOS);
135 scheduler.init(CONTEXT);
136 scheduler.start();
137 for (CallRunner task : tasks) {
138 when(qosFunction.getPriority((RPCProtos.RequestHeader) anyObject(), (Message) anyObject()))
139 .thenReturn(qos.get(task));
140 scheduler.dispatch(task);
141 }
142 for (CallRunner task : tasks) {
143 verify(task, timeout(1000)).run();
144 }
145 scheduler.stop();
146
147
148 countDownLatch.await();
149 assertEquals(3, ImmutableSet.copyOf(handlerThreads.values()).size());
150 }
151
152 private CallRunner createMockTask() {
153 Call call = mock(Call.class);
154 CallRunner task = mock(CallRunner.class);
155 when(task.getCall()).thenReturn(call);
156 return task;
157 }
158
159 @Test
160 public void testRpcScheduler() throws Exception {
161 testRpcScheduler(SimpleRpcScheduler.CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
162 testRpcScheduler(SimpleRpcScheduler.CALL_QUEUE_TYPE_FIFO_CONF_VALUE);
163 }
164
165 private void testRpcScheduler(final String queueType) throws Exception {
166 Configuration schedConf = HBaseConfiguration.create();
167 schedConf.set(SimpleRpcScheduler.CALL_QUEUE_TYPE_CONF_KEY, queueType);
168
169 PriorityFunction priority = mock(PriorityFunction.class);
170 when(priority.getPriority(any(RequestHeader.class), any(Message.class)))
171 .thenReturn(HConstants.NORMAL_QOS);
172
173 RpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 1, 1, 1, priority,
174 HConstants.QOS_THRESHOLD);
175 try {
176 scheduler.start();
177
178 CallRunner smallCallTask = mock(CallRunner.class);
179 RpcServer.Call smallCall = mock(RpcServer.Call.class);
180 RequestHeader smallHead = RequestHeader.newBuilder().setCallId(1).build();
181 when(smallCallTask.getCall()).thenReturn(smallCall);
182 when(smallCall.getHeader()).thenReturn(smallHead);
183
184 CallRunner largeCallTask = mock(CallRunner.class);
185 RpcServer.Call largeCall = mock(RpcServer.Call.class);
186 RequestHeader largeHead = RequestHeader.newBuilder().setCallId(50).build();
187 when(largeCallTask.getCall()).thenReturn(largeCall);
188 when(largeCall.getHeader()).thenReturn(largeHead);
189
190 CallRunner hugeCallTask = mock(CallRunner.class);
191 RpcServer.Call hugeCall = mock(RpcServer.Call.class);
192 RequestHeader hugeHead = RequestHeader.newBuilder().setCallId(100).build();
193 when(hugeCallTask.getCall()).thenReturn(hugeCall);
194 when(hugeCall.getHeader()).thenReturn(hugeHead);
195
196 when(priority.getDeadline(eq(smallHead), any(Message.class))).thenReturn(0L);
197 when(priority.getDeadline(eq(largeHead), any(Message.class))).thenReturn(50L);
198 when(priority.getDeadline(eq(hugeHead), any(Message.class))).thenReturn(100L);
199
200 final ArrayList<Integer> work = new ArrayList<Integer>();
201 doAnswerTaskExecution(smallCallTask, work, 10, 250);
202 doAnswerTaskExecution(largeCallTask, work, 50, 250);
203 doAnswerTaskExecution(hugeCallTask, work, 100, 250);
204
205 scheduler.dispatch(smallCallTask);
206 scheduler.dispatch(smallCallTask);
207 scheduler.dispatch(smallCallTask);
208 scheduler.dispatch(hugeCallTask);
209 scheduler.dispatch(smallCallTask);
210 scheduler.dispatch(largeCallTask);
211 scheduler.dispatch(smallCallTask);
212 scheduler.dispatch(smallCallTask);
213
214 while (work.size() < 8) {
215 Threads.sleepWithoutInterrupt(100);
216 }
217
218 int seqSum = 0;
219 int totalTime = 0;
220 for (int i = 0; i < work.size(); ++i) {
221 LOG.debug("Request i=" + i + " value=" + work.get(i));
222 seqSum += work.get(i);
223 totalTime += seqSum;
224 }
225 LOG.debug("Total Time: " + totalTime);
226
227
228
229
230 if (queueType.equals(SimpleRpcScheduler.CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
231 assertEquals(530, totalTime);
232 } else
233 assertEquals(930, totalTime);
234 }
235 } finally {
236 scheduler.stop();
237 }
238 }
239
240 @Test
241 public void testScanQueueWithZeroScanRatio() throws Exception {
242 Configuration schedConf = HBaseConfiguration.create();
243 schedConf.setFloat(SimpleRpcScheduler.CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 1.0f);
244 schedConf.setFloat(SimpleRpcScheduler.CALL_QUEUE_READ_SHARE_CONF_KEY, 0.5f);
245 schedConf.setFloat(SimpleRpcScheduler.CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0f);
246
247 PriorityFunction priority = mock(PriorityFunction.class);
248
249 RpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 2, 1, 1, priority,
250 HConstants.QOS_THRESHOLD);
251 assertNotEquals(scheduler, null);
252 }
253
254 @Test
255 public void testScanQueues() throws Exception {
256 Configuration schedConf = HBaseConfiguration.create();
257 schedConf.setFloat(SimpleRpcScheduler.CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 1.0f);
258 schedConf.setFloat(SimpleRpcScheduler.CALL_QUEUE_READ_SHARE_CONF_KEY, 0.7f);
259 schedConf.setFloat(SimpleRpcScheduler.CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0.5f);
260
261 PriorityFunction priority = mock(PriorityFunction.class);
262 when(priority.getPriority(any(RequestHeader.class), any(Message.class)))
263 .thenReturn(HConstants.NORMAL_QOS);
264
265 RpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 3, 1, 1, priority,
266 HConstants.QOS_THRESHOLD);
267 try {
268 scheduler.start();
269
270 CallRunner putCallTask = mock(CallRunner.class);
271 RpcServer.Call putCall = mock(RpcServer.Call.class);
272 putCall.param = RequestConverter.buildMutateRequest(
273 Bytes.toBytes("abc"), new Put(Bytes.toBytes("row")));
274 RequestHeader putHead = RequestHeader.newBuilder().setMethodName("mutate").build();
275 when(putCallTask.getCall()).thenReturn(putCall);
276 when(putCall.getHeader()).thenReturn(putHead);
277
278 CallRunner getCallTask = mock(CallRunner.class);
279 RpcServer.Call getCall = mock(RpcServer.Call.class);
280 RequestHeader getHead = RequestHeader.newBuilder().setMethodName("get").build();
281 when(getCallTask.getCall()).thenReturn(getCall);
282 when(getCall.getHeader()).thenReturn(getHead);
283
284 CallRunner scanCallTask = mock(CallRunner.class);
285 RpcServer.Call scanCall = mock(RpcServer.Call.class);
286 scanCall.param = ScanRequest.newBuilder().setScannerId(1).build();
287 RequestHeader scanHead = RequestHeader.newBuilder().setMethodName("scan").build();
288 when(scanCallTask.getCall()).thenReturn(scanCall);
289 when(scanCall.getHeader()).thenReturn(scanHead);
290
291 ArrayList<Integer> work = new ArrayList<Integer>();
292 doAnswerTaskExecution(putCallTask, work, 1, 1000);
293 doAnswerTaskExecution(getCallTask, work, 2, 1000);
294 doAnswerTaskExecution(scanCallTask, work, 3, 1000);
295
296
297
298 scheduler.dispatch(putCallTask);
299 scheduler.dispatch(putCallTask);
300 scheduler.dispatch(putCallTask);
301 scheduler.dispatch(getCallTask);
302 scheduler.dispatch(getCallTask);
303 scheduler.dispatch(getCallTask);
304 scheduler.dispatch(scanCallTask);
305 scheduler.dispatch(scanCallTask);
306 scheduler.dispatch(scanCallTask);
307
308 while (work.size() < 6) {
309 Threads.sleepWithoutInterrupt(100);
310 }
311
312 for (int i = 0; i < work.size() - 2; i += 3) {
313 assertNotEquals(work.get(i + 0), work.get(i + 1));
314 assertNotEquals(work.get(i + 0), work.get(i + 2));
315 assertNotEquals(work.get(i + 1), work.get(i + 2));
316 }
317 } finally {
318 scheduler.stop();
319 }
320 }
321
322 private void doAnswerTaskExecution(final CallRunner callTask,
323 final ArrayList<Integer> results, final int value, final int sleepInterval) {
324 callTask.setStatus(new MonitoredRPCHandlerImpl());
325 doAnswer(new Answer<Object>() {
326 @Override
327 public Object answer(InvocationOnMock invocation) {
328 synchronized (results) {
329 results.add(value);
330 }
331 Threads.sleepWithoutInterrupt(sleepInterval);
332 return null;
333 }
334 }).when(callTask).run();
335 }
336 }