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.assertNotNull;
23 import static org.junit.Assert.assertTrue;
24
25 import java.io.IOException;
26 import java.net.InetSocketAddress;
27 import java.util.ArrayList;
28 import java.util.HashMap;
29 import java.util.List;
30 import java.util.Random;
31 import java.util.concurrent.Callable;
32 import java.util.concurrent.atomic.AtomicBoolean;
33 import java.util.concurrent.atomic.AtomicReference;
34 import java.util.concurrent.locks.ReadWriteLock;
35 import java.util.concurrent.locks.ReentrantReadWriteLock;
36
37 import org.apache.commons.logging.Log;
38 import org.apache.commons.logging.LogFactory;
39 import org.apache.hadoop.conf.Configuration;
40 import org.apache.hadoop.hbase.CellScanner;
41 import org.apache.hadoop.hbase.HBaseConfiguration;
42 import org.apache.hadoop.hbase.HConstants;
43 import org.apache.hadoop.hbase.codec.Codec;
44 import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
45 import org.apache.hadoop.hbase.ipc.RpcClient;
46 import org.apache.hadoop.hbase.ipc.RpcScheduler;
47 import org.apache.hadoop.hbase.ipc.RpcServer;
48 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
49 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
50 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto;
51 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
52 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
53 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
54 import org.apache.hadoop.hbase.security.User;
55 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
56 import org.apache.hadoop.hbase.util.Pair;
57 import org.apache.hadoop.hbase.util.Threads;
58 import org.junit.Ignore;
59 import org.junit.Test;
60 import org.junit.experimental.categories.Category;
61 import com.google.common.collect.Lists;
62 import com.google.protobuf.BlockingService;
63 import com.google.protobuf.Message;
64 import com.google.protobuf.RpcController;
65 import com.google.protobuf.ServiceException;
66 import com.google.protobuf.Descriptors.MethodDescriptor;
67
68 @Category(IntegrationTests.class)
69 public class IntegrationTestRpcClient {
70
71 private static final Log LOG = LogFactory.getLog(IntegrationTestRpcClient.class);
72
73 private final Configuration conf;
74
75 private int numIterations = 10;
76
77 public IntegrationTestRpcClient() {
78 conf = HBaseConfiguration.create();
79 }
80
81 static class TestRpcServer extends RpcServer {
82
83 TestRpcServer(Configuration conf) throws IOException {
84 this(new FifoRpcScheduler(conf, 1), conf);
85 }
86
87 TestRpcServer(RpcScheduler scheduler, Configuration conf) throws IOException {
88 super(null, "testRpcServer", Lists
89 .newArrayList(new BlockingServiceAndInterface(SERVICE, null)), new InetSocketAddress(
90 "localhost", 0), conf, scheduler);
91 }
92
93 @Override
94 public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
95 Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
96 throws IOException {
97 return super.call(service, md, param, cellScanner, receiveTime, status);
98 }
99 }
100
101 static final BlockingService SERVICE =
102 TestRpcServiceProtos.TestProtobufRpcProto
103 .newReflectiveBlockingService(new TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface() {
104
105 @Override
106 public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request)
107 throws ServiceException {
108 return null;
109 }
110
111 @Override
112 public EmptyResponseProto error(RpcController controller, EmptyRequestProto request)
113 throws ServiceException {
114 return null;
115 }
116
117 @Override
118 public EchoResponseProto echo(RpcController controller, EchoRequestProto request)
119 throws ServiceException {
120 return EchoResponseProto.newBuilder().setMessage(request.getMessage()).build();
121 }
122 });
123
124 protected RpcClient createRpcClient(Configuration conf, boolean isSyncClient) {
125 return isSyncClient ?
126 new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT) :
127 null;
128 }
129
130 static String BIG_PAYLOAD;
131
132 static {
133 StringBuilder builder = new StringBuilder();
134
135 while (builder.length() < 1024 * 1024) {
136 builder.append("big.payload.");
137 }
138
139 BIG_PAYLOAD = builder.toString();
140 }
141
142 class Cluster {
143 Random random = new Random();
144 ReadWriteLock lock = new ReentrantReadWriteLock();
145 HashMap<InetSocketAddress, TestRpcServer> rpcServers = new HashMap<InetSocketAddress, TestRpcServer>();
146 List<TestRpcServer> serverList = new ArrayList<TestRpcServer>();
147 int maxServers;
148 int minServers;
149
150 Cluster(int minServers, int maxServers) {
151 this.minServers = minServers;
152 this.maxServers = maxServers;
153 }
154
155 TestRpcServer startServer() throws IOException {
156 lock.writeLock().lock();
157 try {
158 if (rpcServers.size() >= maxServers) {
159 return null;
160 }
161
162 TestRpcServer rpcServer = new TestRpcServer(conf);
163 rpcServer.start();
164 InetSocketAddress address = rpcServer.getListenerAddress();
165 if (address == null) {
166 throw new IOException("Listener channel is closed");
167 }
168 rpcServers.put(address, rpcServer);
169 serverList.add(rpcServer);
170 LOG.info("Started server: " + address);
171 return rpcServer;
172 } finally {
173 lock.writeLock().unlock();
174 }
175 }
176
177 void stopRandomServer() throws Exception {
178 lock.writeLock().lock();
179 TestRpcServer rpcServer = null;
180 try {
181 if (rpcServers.size() <= minServers) {
182 return;
183 }
184 int size = rpcServers.size();
185 int rand = random.nextInt(size);
186 rpcServer = serverList.remove(rand);
187 InetSocketAddress address = rpcServer.getListenerAddress();
188 if (address == null) {
189
190
191 throw new IOException("Listener channel is closed");
192 }
193 rpcServers.remove(address);
194
195 if (rpcServer != null) {
196 stopServer(rpcServer);
197 }
198 } finally {
199 lock.writeLock().unlock();
200 }
201 }
202
203 void stopServer(TestRpcServer rpcServer) throws InterruptedException {
204 InetSocketAddress address = rpcServer.getListenerAddress();
205 LOG.info("Stopping server: " + address);
206 rpcServer.stop();
207 rpcServer.join();
208 LOG.info("Stopped server: " + address);
209 }
210
211 void stopRunning() throws InterruptedException {
212 lock.writeLock().lock();
213 try {
214 for (TestRpcServer rpcServer : serverList) {
215 stopServer(rpcServer);
216 }
217
218 } finally {
219 lock.writeLock().unlock();
220 }
221 }
222
223 TestRpcServer getRandomServer() {
224 lock.readLock().lock();
225 try {
226 int size = rpcServers.size();
227 int rand = random.nextInt(size);
228 return serverList.get(rand);
229 } finally {
230 lock.readLock().unlock();
231 }
232 }
233 }
234
235 static class MiniChaosMonkey extends Thread {
236 AtomicBoolean running = new AtomicBoolean(true);
237 Random random = new Random();
238 AtomicReference<Exception> exception = new AtomicReference<Exception>(null);
239 Cluster cluster;
240
241 public MiniChaosMonkey(Cluster cluster) {
242 this.cluster = cluster;
243 }
244
245 @Override
246 public void run() {
247 while (running.get()) {
248 switch (random.nextInt() % 2) {
249 case 0:
250 try {
251 cluster.startServer();
252 } catch (Exception e) {
253 LOG.warn(e);
254 exception.compareAndSet(null, e);
255 }
256 break;
257
258 case 1:
259 try {
260 cluster.stopRandomServer();
261 } catch (Exception e) {
262 LOG.warn(e);
263 exception.compareAndSet(null, e);
264 }
265 default:
266 }
267
268 Threads.sleep(100);
269 }
270 }
271
272 void stopRunning() {
273 running.set(false);
274 }
275
276 void rethrowException() throws Exception {
277 if (exception.get() != null) {
278 throw exception.get();
279 }
280 }
281 }
282
283 static class SimpleClient extends Thread {
284 RpcClient rpcClient;
285 AtomicBoolean running = new AtomicBoolean(true);
286 AtomicReference<Throwable> exception = new AtomicReference<Throwable>(null);
287 Cluster cluster;
288 String id;
289 long numCalls = 0;
290 Random random = new Random();
291
292 public SimpleClient(Cluster cluster, RpcClient rpcClient, String id) {
293 this.cluster = cluster;
294 this.rpcClient = rpcClient;
295 this.id = id;
296 }
297
298 @Override
299 public void run() {
300 MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo");
301
302 while (running.get()) {
303 boolean isBigPayload = random.nextBoolean();
304 String message = isBigPayload ? BIG_PAYLOAD : id + numCalls;
305 EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message).build();
306 EchoResponseProto ret = EchoResponseProto.newBuilder().setMessage("foo").build();
307
308 TestRpcServer server = cluster.getRandomServer();
309 try {
310 User user = User.getCurrent();
311 InetSocketAddress address = server.getListenerAddress();
312 if (address == null) {
313 throw new IOException("Listener channel is closed");
314 }
315 ret = (EchoResponseProto)
316 rpcClient.callBlockingMethod(md, null, param, ret, user, address, 60000);
317 } catch (Exception e) {
318 LOG.warn(e);
319 continue;
320 }
321
322 try {
323 assertNotNull(ret);
324 assertEquals(message, ret.getMessage());
325 } catch (Throwable t) {
326 exception.compareAndSet(null, t);
327 }
328
329 numCalls++;
330 }
331 }
332
333 void stopRunning() {
334 running.set(false);
335 }
336
337 void rethrowException() throws Throwable {
338 if (exception.get() != null) {
339 throw exception.get();
340 }
341 }
342 }
343
344 @Test (timeout = 900000)
345 public void testRpcWithChaosMonkeyWithSyncClient() throws Throwable {
346 for (int i = 0; i < numIterations; i++) {
347 TimeoutThread.runWithTimeout(new Callable<Void>() {
348 @Override
349 public Void call() throws Exception {
350 try {
351 testRpcWithChaosMonkey(true);
352 } catch (Throwable e) {
353 if (e instanceof Exception) {
354 throw (Exception)e;
355 } else {
356 throw new Exception(e);
357 }
358 }
359 return null;
360 }
361 }, 90000);
362 }
363 }
364
365 static class TimeoutThread extends Thread {
366 long timeout;
367 public TimeoutThread(long timeout) {
368 this.timeout = timeout;
369 }
370
371 @Override
372 public void run() {
373 try {
374 Thread.sleep(timeout);
375 Threads.printThreadInfo(System.err, "TEST TIMEOUT STACK DUMP");
376 System.exit(1);
377 } catch (InterruptedException e) {
378
379 }
380 }
381
382
383
384 static void runWithTimeout(Callable<?> callable, long timeout) throws Exception {
385 TimeoutThread thread = new TimeoutThread(timeout);
386 thread.start();
387 callable.call();
388 thread.interrupt();
389 }
390 }
391
392 public void testRpcWithChaosMonkey(boolean isSyncClient) throws Throwable {
393 LOG.info("Starting test");
394 Cluster cluster = new Cluster(10, 100);
395 for (int i = 0; i < 10; i++) {
396 cluster.startServer();
397 }
398
399 ArrayList<SimpleClient> clients = new ArrayList<SimpleClient>();
400
401
402 RpcClient rpcClient = createRpcClient(conf, isSyncClient);
403
404 for (int i = 0; i < 30; i++) {
405 String clientId = "client_" + i + "_";
406 LOG.info("Starting client: " + clientId);
407 SimpleClient client = new SimpleClient(cluster, rpcClient, clientId);
408 client.start();
409 clients.add(client);
410 }
411
412 LOG.info("Starting MiniChaosMonkey");
413 MiniChaosMonkey cm = new MiniChaosMonkey(cluster);
414 cm.start();
415
416 Threads.sleep(30000);
417
418 LOG.info("Stopping MiniChaosMonkey");
419 cm.stopRunning();
420 cm.join();
421 cm.rethrowException();
422
423 LOG.info("Stopping clients");
424 for (SimpleClient client : clients) {
425 LOG.info("Stopping client: " + client.id);
426 LOG.info(client.id + " numCalls:" + client.numCalls);
427 client.stopRunning();
428 client.join();
429 client.rethrowException();
430 assertTrue(client.numCalls > 10);
431 }
432
433 LOG.info("Stopping RpcClient");
434 rpcClient.stop();
435
436 LOG.info("Stopping Cluster");
437 cluster.stopRunning();
438 }
439 }