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 static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertFalse;
23 import static org.junit.Assert.assertTrue;
24 import static org.junit.Assert.fail;
25
26 import java.io.IOException;
27 import java.net.InetSocketAddress;
28 import java.util.ArrayList;
29 import java.util.List;
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.HBaseConfiguration;
35 import org.apache.hadoop.hbase.HConstants;
36 import org.apache.hadoop.hbase.testclassification.MediumTests;
37 import org.apache.hadoop.hbase.ServerName;
38 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos;
39 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg;
40 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse;
41 import org.apache.hadoop.hbase.security.User;
42 import org.apache.log4j.AppenderSkeleton;
43 import org.apache.log4j.Level;
44 import org.apache.log4j.Logger;
45 import org.apache.log4j.spi.LoggingEvent;
46 import org.junit.Test;
47 import org.junit.experimental.categories.Category;
48
49 import com.google.common.collect.Lists;
50 import com.google.protobuf.BlockingRpcChannel;
51 import com.google.protobuf.BlockingService;
52 import com.google.protobuf.RpcController;
53 import com.google.protobuf.ServiceException;
54
55
56
57
58
59
60 @Category(MediumTests.class)
61 public class TestDelayedRpc {
62 private static final Log LOG = LogFactory.getLog(TestDelayedRpc.class);
63 public static RpcServerInterface rpcServer;
64 public static final int UNDELAYED = 0;
65 public static final int DELAYED = 1;
66 private static final int RPC_CLIENT_TIMEOUT = 30000;
67
68 @Test (timeout=60000)
69 public void testDelayedRpcImmediateReturnValue() throws Exception {
70 testDelayedRpc(false);
71 }
72
73 @Test (timeout=60000)
74 public void testDelayedRpcDelayedReturnValue() throws Exception {
75 testDelayedRpc(true);
76 }
77
78 private void testDelayedRpc(boolean delayReturnValue) throws Exception {
79 LOG.info("Running testDelayedRpc delayReturnValue=" + delayReturnValue);
80 Configuration conf = HBaseConfiguration.create();
81 InetSocketAddress isa = new InetSocketAddress("localhost", 0);
82 TestDelayedImplementation instance = new TestDelayedImplementation(delayReturnValue);
83 BlockingService service =
84 TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
85 rpcServer = new RpcServer(null, "testDelayedRpc",
86 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)),
87 isa,
88 conf,
89 new FifoRpcScheduler(conf, 1));
90 rpcServer.start();
91 RpcClient rpcClient = RpcClientFactory.createClient(
92 conf, HConstants.DEFAULT_CLUSTER_ID.toString());
93 try {
94 InetSocketAddress address = rpcServer.getListenerAddress();
95 if (address == null) {
96 throw new IOException("Listener channel is closed");
97 }
98 BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
99 ServerName.valueOf(address.getHostName(),address.getPort(), System.currentTimeMillis()),
100 User.getCurrent(), RPC_CLIENT_TIMEOUT);
101 TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
102 TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
103 List<Integer> results = new ArrayList<Integer>();
104
105 TestThread th1 = new TestThread(stub, true, results);
106
107 TestThread th2 = new TestThread(stub, false, results);
108 TestThread th3 = new TestThread(stub, false, results);
109 th1.start();
110 Thread.sleep(100);
111 th2.start();
112 Thread.sleep(200);
113 th3.start();
114
115 th1.join();
116 th2.join();
117 th3.join();
118
119
120 assertEquals(UNDELAYED, results.get(0).intValue());
121 assertEquals(UNDELAYED, results.get(1).intValue());
122 assertEquals(results.get(2).intValue(), delayReturnValue ? DELAYED : 0xDEADBEEF);
123 } finally {
124 rpcClient.close();
125 }
126 }
127
128 private static class ListAppender extends AppenderSkeleton {
129 private final List<String> messages = new ArrayList<String>();
130
131 @Override
132 protected void append(LoggingEvent event) {
133 messages.add(event.getMessage().toString());
134 }
135
136 @Override
137 public void close() {
138 }
139
140 @Override
141 public boolean requiresLayout() {
142 return false;
143 }
144
145 public List<String> getMessages() {
146 return messages;
147 }
148 }
149
150
151
152
153
154 @Test (timeout=60000)
155 public void testTooManyDelayedRpcs() throws Exception {
156 Configuration conf = HBaseConfiguration.create();
157 final int MAX_DELAYED_RPC = 10;
158 conf.setInt("hbase.ipc.warn.delayedrpc.number", MAX_DELAYED_RPC);
159
160 ListAppender listAppender = new ListAppender();
161 Logger log = Logger.getLogger(RpcServer.class);
162 log.addAppender(listAppender);
163 log.setLevel(Level.WARN);
164
165
166 InetSocketAddress isa = new InetSocketAddress("localhost", 0);
167 TestDelayedImplementation instance = new TestDelayedImplementation(true);
168 BlockingService service =
169 TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
170 rpcServer = new RpcServer(null, "testTooManyDelayedRpcs",
171 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)),
172 isa,
173 conf,
174 new FifoRpcScheduler(conf, 1));
175 rpcServer.start();
176 RpcClient rpcClient = RpcClientFactory.createClient(
177 conf, HConstants.DEFAULT_CLUSTER_ID.toString());
178 try {
179 InetSocketAddress address = rpcServer.getListenerAddress();
180 if (address == null) {
181 throw new IOException("Listener channel is closed");
182 }
183 BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
184 ServerName.valueOf(address.getHostName(),address.getPort(), System.currentTimeMillis()),
185 User.getCurrent(), RPC_CLIENT_TIMEOUT);
186 TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
187 TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
188 Thread threads[] = new Thread[MAX_DELAYED_RPC + 1];
189 for (int i = 0; i < MAX_DELAYED_RPC; i++) {
190 threads[i] = new TestThread(stub, true, null);
191 threads[i].start();
192 }
193
194
195 assertTrue(listAppender.getMessages().isEmpty());
196
197
198 threads[MAX_DELAYED_RPC] = new TestThread(stub, true, null);
199 threads[MAX_DELAYED_RPC].start();
200
201 for (int i = 0; i < MAX_DELAYED_RPC; i++) {
202 threads[i].join();
203 }
204
205 assertFalse(listAppender.getMessages().isEmpty());
206 assertTrue(listAppender.getMessages().get(0).startsWith("Too many delayed calls"));
207
208 log.removeAppender(listAppender);
209 } finally {
210 rpcClient.close();
211 }
212 }
213
214 public static class TestDelayedImplementation
215 implements TestDelayedRpcProtos.TestDelayedService.BlockingInterface {
216
217
218
219
220 private final boolean delayReturnValue;
221
222
223
224
225
226 public TestDelayedImplementation(boolean delayReturnValue) {
227 this.delayReturnValue = delayReturnValue;
228 }
229
230 @Override
231 public TestResponse test(final RpcController rpcController, final TestArg testArg)
232 throws ServiceException {
233 boolean delay = testArg.getDelay();
234 TestResponse.Builder responseBuilder = TestResponse.newBuilder();
235 if (!delay) {
236 responseBuilder.setResponse(UNDELAYED);
237 return responseBuilder.build();
238 }
239 final Delayable call = RpcServer.getCurrentCall();
240 call.startDelay(delayReturnValue);
241 new Thread() {
242 @Override
243 public void run() {
244 try {
245 Thread.sleep(500);
246 TestResponse.Builder responseBuilder = TestResponse.newBuilder();
247 call.endDelay(delayReturnValue ?
248 responseBuilder.setResponse(DELAYED).build() : null);
249 } catch (Exception e) {
250 e.printStackTrace();
251 }
252 }
253 }.start();
254
255
256 responseBuilder.setResponse(0xDEADBEEF);
257 return responseBuilder.build();
258 }
259 }
260
261 public static class TestThread extends Thread {
262 private final TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub;
263 private final boolean delay;
264 private final List<Integer> results;
265
266 public TestThread(TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub,
267 boolean delay, List<Integer> results) {
268 this.stub = stub;
269 this.delay = delay;
270 this.results = results;
271 }
272
273 @Override
274 public void run() {
275 Integer result;
276 try {
277 result = new Integer(stub.test(null, TestArg.newBuilder().setDelay(delay).build()).
278 getResponse());
279 } catch (ServiceException e) {
280 throw new RuntimeException(e);
281 }
282 if (results != null) {
283 synchronized (results) {
284 results.add(result);
285 }
286 }
287 }
288 }
289
290 @Test
291 public void testEndDelayThrowing() throws IOException {
292 Configuration conf = HBaseConfiguration.create();
293 InetSocketAddress isa = new InetSocketAddress("localhost", 0);
294 FaultyTestDelayedImplementation instance = new FaultyTestDelayedImplementation();
295 BlockingService service =
296 TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
297 rpcServer = new RpcServer(null, "testEndDelayThrowing",
298 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)),
299 isa,
300 conf,
301 new FifoRpcScheduler(conf, 1));
302 rpcServer.start();
303 RpcClient rpcClient = RpcClientFactory.createClient(
304 conf, HConstants.DEFAULT_CLUSTER_ID.toString());
305 try {
306 InetSocketAddress address = rpcServer.getListenerAddress();
307 if (address == null) {
308 throw new IOException("Listener channel is closed");
309 }
310 BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
311 ServerName.valueOf(address.getHostName(),address.getPort(), System.currentTimeMillis()),
312 User.getCurrent(), 1000);
313 TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
314 TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
315
316 int result = 0xDEADBEEF;
317
318 try {
319 result = stub.test(null, TestArg.newBuilder().setDelay(false).build()).getResponse();
320 } catch (Exception e) {
321 fail("No exception should have been thrown.");
322 }
323 assertEquals(result, UNDELAYED);
324
325 boolean caughtException = false;
326 try {
327 result = stub.test(null, TestArg.newBuilder().setDelay(true).build()).getResponse();
328 } catch(Exception e) {
329
330 if (e.getCause().getMessage().contains("java.lang.Exception: Something went wrong")) {
331 caughtException = true;
332 }
333 LOG.warn("Caught exception, expected=" + caughtException);
334 }
335 assertTrue(caughtException);
336 } finally {
337 rpcClient.close();
338 }
339 }
340
341
342
343
344 private static class FaultyTestDelayedImplementation extends TestDelayedImplementation {
345 public FaultyTestDelayedImplementation() {
346 super(false);
347 }
348
349 @Override
350 public TestResponse test(RpcController rpcController, TestArg arg)
351 throws ServiceException {
352 LOG.info("In faulty test, delay=" + arg.getDelay());
353 if (!arg.getDelay()) return TestResponse.newBuilder().setResponse(UNDELAYED).build();
354 Delayable call = RpcServer.getCurrentCall();
355 call.startDelay(true);
356 LOG.info("In faulty test, delaying");
357 try {
358 call.endDelayThrowing(new Exception("Something went wrong"));
359 } catch (IOException e) {
360 e.printStackTrace();
361 }
362
363 return TestResponse.newBuilder().setResponse(DELAYED).build();
364 }
365 }
366 }