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