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 static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE;
21  import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub;
22  import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newStub;
23  import static org.junit.Assert.assertEquals;
24  import static org.junit.Assert.assertFalse;
25  import static org.junit.Assert.assertNotNull;
26  import static org.junit.Assert.assertNull;
27  import static org.junit.Assert.assertTrue;
28  import static org.junit.Assert.fail;
29  import static org.mockito.Matchers.anyObject;
30  import static org.mockito.Mockito.spy;
31  import static org.mockito.Mockito.verify;
32  import static org.mockito.internal.verification.VerificationModeFactory.times;
33  
34  import com.google.common.collect.ImmutableList;
35  import com.google.common.collect.Lists;
36  import com.google.protobuf.ServiceException;
37  
38  import java.io.IOException;
39  import java.net.InetSocketAddress;
40  import java.nio.ByteBuffer;
41  import java.nio.channels.SocketChannel;
42  import java.util.ArrayList;
43  import java.util.List;
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.Cell;
49  import org.apache.hadoop.hbase.CellScanner;
50  import org.apache.hadoop.hbase.CellUtil;
51  import org.apache.hadoop.hbase.DoNotRetryIOException;
52  import org.apache.hadoop.hbase.HBaseConfiguration;
53  import org.apache.hadoop.hbase.KeyValue;
54  import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
55  import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto;
56  import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
57  import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
58  import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto;
59  import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
60  import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface;
61  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
62  import org.apache.hadoop.hbase.util.Bytes;
63  import org.apache.hadoop.io.compress.GzipCodec;
64  import org.apache.hadoop.util.StringUtils;
65  import org.junit.Test;
66  
67  /**
68   * Some basic ipc tests.
69   */
70  public abstract class AbstractTestIPC {
71  
72    private static final Log LOG = LogFactory.getLog(AbstractTestIPC.class);
73  
74    private static final byte[] CELL_BYTES = Bytes.toBytes("xyz");
75    private static final KeyValue CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES);
76    static byte[] BIG_CELL_BYTES = new byte[10 * 1024];
77    static KeyValue BIG_CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, BIG_CELL_BYTES);
78    static final Configuration CONF = HBaseConfiguration.create();
79  
80    /**
81     * Instance of server. We actually don't do anything speical in here so could just use
82     * HBaseRpcServer directly.
83     */
84    static class TestRpcServer extends RpcServer {
85  
86      TestRpcServer() throws IOException {
87        this(new FifoRpcScheduler(CONF, 1), CONF);
88      }
89  
90      TestRpcServer(Configuration conf) throws IOException {
91        this(new FifoRpcScheduler(conf, 1), conf);
92      }
93  
94      TestRpcServer(RpcScheduler scheduler, Configuration conf) throws IOException {
95        super(null, "testRpcServer",
96            Lists.newArrayList(new BlockingServiceAndInterface(SERVICE, null)),
97            new InetSocketAddress("localhost", 0), conf, scheduler);
98      }
99    }
100 
101   protected abstract AbstractRpcClient<?> createRpcClientNoCodec(Configuration conf);
102 
103   /**
104    * Ensure we do not HAVE TO HAVE a codec.
105    */
106   @Test
107   public void testNoCodec() throws IOException, ServiceException {
108     Configuration conf = HBaseConfiguration.create();
109     TestRpcServer rpcServer = new TestRpcServer();
110     try (AbstractRpcClient<?> client = createRpcClientNoCodec(conf)) {
111       rpcServer.start();
112       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
113       HBaseRpcController pcrc = new HBaseRpcControllerImpl();
114       String message = "hello";
115       assertEquals(message,
116         stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage(message).build()).getMessage());
117       assertNull(pcrc.cellScanner());
118     } finally {
119       rpcServer.stop();
120     }
121   }
122 
123   protected abstract AbstractRpcClient<?> createRpcClient(Configuration conf);
124 
125   /**
126    * It is hard to verify the compression is actually happening under the wraps. Hope that if
127    * unsupported, we'll get an exception out of some time (meantime, have to trace it manually to
128    * confirm that compression is happening down in the client and server).
129    */
130   @Test
131   public void testCompressCellBlock() throws IOException, ServiceException {
132     Configuration conf = new Configuration(HBaseConfiguration.create());
133    // conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName());
134     List<Cell> cells = new ArrayList<>();
135     int count = 3;
136     for (int i = 0; i < count; i++) {
137       cells.add(CELL);
138     }
139     TestRpcServer rpcServer = new TestRpcServer();
140     try (AbstractRpcClient<?> client = createRpcClient(conf)) {
141       rpcServer.start();
142       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
143       HBaseRpcController pcrc = new HBaseRpcControllerImpl(CellUtil.createCellScanner(cells));
144       String message = "hello";
145       assertEquals(message,
146         stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage(message).build()).getMessage());
147       int index = 0;
148       CellScanner cellScanner = pcrc.cellScanner();
149       assertNotNull(cellScanner);
150       while (cellScanner.advance()) {
151         assertEquals(CELL, cellScanner.current());
152         index++;
153       }
154       assertEquals(count, index);
155     } finally {
156       rpcServer.stop();
157     }
158   }
159 
160   protected abstract AbstractRpcClient<?> createRpcClientRTEDuringConnectionSetup(
161       Configuration conf) throws IOException;
162 
163   @Test
164   public void testRTEDuringConnectionSetup() throws Exception {
165     Configuration conf = HBaseConfiguration.create();
166     TestRpcServer rpcServer = new TestRpcServer();
167     try (AbstractRpcClient<?> client = createRpcClientRTEDuringConnectionSetup(conf)) {
168       rpcServer.start();
169       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
170       stub.ping(null, EmptyRequestProto.getDefaultInstance());
171       fail("Expected an exception to have been thrown!");
172     } catch (Exception e) {
173       LOG.info("Caught expected exception: " + e.toString());
174       assertTrue(e.toString(), StringUtils.stringifyException(e).contains("Injected fault"));
175     } finally {
176       rpcServer.stop();
177     }
178   }
179 
180   /**
181    * Tests that the rpc scheduler is called when requests arrive.
182    */
183   @Test
184   public void testRpcScheduler() throws IOException, ServiceException, InterruptedException {
185     RpcScheduler scheduler = spy(new FifoRpcScheduler(CONF, 1));
186     RpcServer rpcServer = new TestRpcServer(scheduler, CONF);
187     verify(scheduler).init((RpcScheduler.Context) anyObject());
188     try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
189       rpcServer.start();
190       verify(scheduler).start();
191       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
192       EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build();
193       for (int i = 0; i < 10; i++) {
194         stub.echo(null, param);
195       }
196       verify(scheduler, times(10)).dispatch((CallRunner) anyObject());
197     } finally {
198       rpcServer.stop();
199       verify(scheduler).stop();
200     }
201   }
202 
203   /** Tests that the rpc scheduler is called when requests arrive. */
204   @Test
205   public void testRpcMaxRequestSize() throws IOException, ServiceException {
206     Configuration conf = new Configuration(CONF);
207     conf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000);
208     RpcServer rpcServer = new TestRpcServer(conf);
209     try (AbstractRpcClient<?> client = createRpcClient(conf)) {
210       rpcServer.start();
211       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
212       StringBuilder message = new StringBuilder(1200);
213       for (int i = 0; i < 200; i++) {
214         message.append("hello.");
215       }
216       // set total RPC size bigger than 100 bytes
217       EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message.toString()).build();
218       stub.echo(
219         new HBaseRpcControllerImpl(CellUtil.createCellScanner(ImmutableList.<Cell> of(CELL))),
220         param);
221       fail("RPC should have failed because it exceeds max request size");
222     } catch (ServiceException e) {
223       LOG.info("Caught expected exception: " + e);
224       assertTrue(e.toString(),
225         StringUtils.stringifyException(e).contains("RequestTooBigException"));
226     } finally {
227       rpcServer.stop();
228     }
229   }
230 
231   /**
232    * Tests that the RpcServer creates & dispatches CallRunner object to scheduler with non-null
233    * remoteAddress set to its Call Object
234    * @throws ServiceException
235    */
236   @Test
237   public void testRpcServerForNotNullRemoteAddressInCallObject()
238       throws IOException, ServiceException {
239     TestRpcServer rpcServer = new TestRpcServer();
240     InetSocketAddress localAddr = new InetSocketAddress("localhost", 0);
241     try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
242       rpcServer.start();
243       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
244       assertEquals(localAddr.getAddress().getHostAddress(),
245         stub.addr(null, EmptyRequestProto.getDefaultInstance()).getAddr());
246     } finally {
247       rpcServer.stop();
248     }
249   }
250 
251   @Test
252   public void testRemoteError() throws IOException, ServiceException {
253     TestRpcServer rpcServer = new TestRpcServer();
254     try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
255       rpcServer.start();
256       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
257       stub.error(null, EmptyRequestProto.getDefaultInstance());
258     } catch (ServiceException e) {
259       LOG.info("Caught expected exception: " + e);
260       IOException ioe = ProtobufUtil.handleRemoteException(e);
261       assertTrue(ioe instanceof DoNotRetryIOException);
262       assertTrue(ioe.getMessage().contains("server error!"));
263     } finally {
264       rpcServer.stop();
265     }
266   }
267 
268   @Test
269   public void testTimeout() throws IOException {
270     TestRpcServer rpcServer = new TestRpcServer();
271     try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
272       rpcServer.start();
273       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
274       HBaseRpcController pcrc = new HBaseRpcControllerImpl();
275       int ms = 1000;
276       int timeout = 100;
277       for (int i = 0; i < 10; i++) {
278         pcrc.reset();
279         pcrc.setCallTimeout(timeout);
280         long startTime = System.nanoTime();
281         try {
282           stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(ms).build());
283         } catch (ServiceException e) {
284           long waitTime = (System.nanoTime() - startTime) / 1000000;
285           // expected
286           LOG.info("Caught expected exception: " + e);
287           IOException ioe = ProtobufUtil.handleRemoteException(e);
288           assertTrue(ioe.getCause() instanceof CallTimeoutException);
289           // confirm that we got exception before the actual pause.
290           assertTrue(waitTime < ms);
291         }
292       }
293     } finally {
294       rpcServer.stop();
295     }
296   }
297 
298   static class TestFailingRpcServer extends TestRpcServer {
299 
300     TestFailingRpcServer() throws IOException {
301       this(new FifoRpcScheduler(CONF, 1), CONF);
302     }
303 
304     TestFailingRpcServer(Configuration conf) throws IOException {
305       this(new FifoRpcScheduler(conf, 1), conf);
306     }
307 
308     TestFailingRpcServer(RpcScheduler scheduler, Configuration conf) throws IOException {
309       super(scheduler, conf);
310     }
311 
312     class FailingConnection extends Connection {
313       public FailingConnection(SocketChannel channel, long lastContact) {
314         super(channel, lastContact);
315       }
316 
317       @Override
318       protected void processRequest(ByteBuffer buf) throws IOException, InterruptedException {
319         // this will throw exception after the connection header is read, and an RPC is sent
320         // from client
321         throw new DoNotRetryIOException("Failing for test");
322       }
323     }
324 
325     @Override
326     protected Connection getConnection(SocketChannel channel, long time) {
327       return new FailingConnection(channel, time);
328     }
329   }
330 
331   /** Tests that the connection closing is handled by the client with outstanding RPC calls */
332   @Test
333   public void testConnectionCloseWithOutstandingRPCs() throws InterruptedException, IOException {
334     Configuration conf = new Configuration(CONF);
335     RpcServer rpcServer = new TestFailingRpcServer(conf);
336     try (AbstractRpcClient<?> client = createRpcClient(conf)) {
337       rpcServer.start();
338       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
339       EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build();
340       stub.echo(null, param);
341       fail("RPC should have failed because connection closed");
342     } catch (ServiceException e) {
343       LOG.info("Caught expected exception: " + e.toString());
344     } finally {
345       rpcServer.stop();
346     }
347   }
348 
349   @Test
350   public void testAsyncEcho() throws IOException {
351     Configuration conf = HBaseConfiguration.create();
352     TestRpcServer rpcServer = new TestRpcServer();
353     try (AbstractRpcClient<?> client = createRpcClient(conf)) {
354       rpcServer.start();
355       Interface stub = newStub(client, rpcServer.getListenerAddress());
356       int num = 10;
357       List<HBaseRpcController> pcrcList = new ArrayList<>();
358       List<BlockingRpcCallback<EchoResponseProto>> callbackList = new ArrayList<>();
359       for (int i = 0; i < num; i++) {
360         HBaseRpcController pcrc = new HBaseRpcControllerImpl();
361         BlockingRpcCallback<EchoResponseProto> done = new BlockingRpcCallback<>();
362         stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage("hello-" + i).build(), done);
363         pcrcList.add(pcrc);
364         callbackList.add(done);
365       }
366       for (int i = 0; i < num; i++) {
367         HBaseRpcController pcrc = pcrcList.get(i);
368         assertFalse(pcrc.failed());
369         assertNull(pcrc.cellScanner());
370         assertEquals("hello-" + i, callbackList.get(i).get().getMessage());
371       }
372     } finally {
373       rpcServer.stop();
374     }
375   }
376 
377   @Test
378   public void testAsyncRemoteError() throws IOException {
379     AbstractRpcClient<?> client = createRpcClient(CONF);
380     TestRpcServer rpcServer = new TestRpcServer();
381     try {
382       rpcServer.start();
383       Interface stub = newStub(client, rpcServer.getListenerAddress());
384       BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
385       HBaseRpcController pcrc = new HBaseRpcControllerImpl();
386       stub.error(pcrc, EmptyRequestProto.getDefaultInstance(), callback);
387       assertNull(callback.get());
388       assertTrue(pcrc.failed());
389       LOG.info("Caught expected exception: " + pcrc.getFailed());
390       IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed());
391       assertTrue(ioe instanceof DoNotRetryIOException);
392       assertTrue(ioe.getMessage().contains("server error!"));
393     } finally {
394       client.close();
395       rpcServer.stop();
396     }
397   }
398 
399   @Test
400   public void testAsyncTimeout() throws IOException {
401     TestRpcServer rpcServer = new TestRpcServer();
402     try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
403       rpcServer.start();
404       Interface stub = newStub(client, rpcServer.getListenerAddress());
405       List<HBaseRpcController> pcrcList = new ArrayList<>();
406       List<BlockingRpcCallback<EmptyResponseProto>> callbackList = new ArrayList<>();
407       int ms = 1000;
408       int timeout = 100;
409       long startTime = System.nanoTime();
410       for (int i = 0; i < 10; i++) {
411         HBaseRpcController pcrc = new HBaseRpcControllerImpl();
412         pcrc.setCallTimeout(timeout);
413         BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
414         stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(ms).build(), callback);
415         pcrcList.add(pcrc);
416         callbackList.add(callback);
417       }
418       for (BlockingRpcCallback<?> callback : callbackList) {
419         assertNull(callback.get());
420       }
421       long waitTime = (System.nanoTime() - startTime) / 1000000;
422       for (HBaseRpcController pcrc : pcrcList) {
423         assertTrue(pcrc.failed());
424         LOG.info("Caught expected exception: " + pcrc.getFailed());
425         IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed());
426         assertTrue(ioe.getCause() instanceof CallTimeoutException);
427       }
428       // confirm that we got exception before the actual pause.
429       assertTrue(waitTime < ms);
430     } finally {
431       rpcServer.stop();
432     }
433   }
434 }