View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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     // Tests that these requests are handled by three distinct threads.
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       // -> [small small small huge small large small small]
228       // -> NO REORDER   [10 10 10 100 10 50 10 10] -> 930 (FIFO Queue)
229       // -> WITH REORDER [10 10 10 10 10 10 50 100] -> 530 (Deadline Queue)
230       if (queueType.equals(SimpleRpcScheduler.CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
231         assertEquals(530, totalTime);
232       } else /* if (queueType.equals(SimpleRpcScheduler.CALL_QUEUE_TYPE_FIFO_CONF_VALUE)) */ {
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       // There are 3 queues: [puts], [gets], [scans]
297       // so the calls will be interleaved
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 }