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