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