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.assertTrue;
23 import static org.junit.Assert.fail;
24 import static org.mockito.Matchers.anyInt;
25 import static org.mockito.Mockito.doThrow;
26 import static org.mockito.Mockito.spy;
27
28 import java.io.IOException;
29 import java.net.InetSocketAddress;
30 import java.net.Socket;
31 import java.util.ArrayList;
32 import java.util.List;
33
34 import javax.net.SocketFactory;
35
36 import com.google.common.collect.Lists;
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.Cell;
41 import org.apache.hadoop.hbase.CellScannable;
42 import org.apache.hadoop.hbase.CellScanner;
43 import org.apache.hadoop.hbase.CellUtil;
44 import org.apache.hadoop.hbase.HBaseConfiguration;
45 import org.apache.hadoop.hbase.HConstants;
46 import org.apache.hadoop.hbase.KeyValue;
47 import org.apache.hadoop.hbase.KeyValueUtil;
48 import org.apache.hadoop.hbase.SmallTests;
49 import org.apache.hadoop.hbase.client.Put;
50 import org.apache.hadoop.hbase.client.RowMutations;
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.TestProtos.EmptyRequestProto;
54 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
55 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
56 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
57 import org.apache.hadoop.hbase.protobuf.RequestConverter;
58 import org.apache.hadoop.hbase.security.User;
59 import org.apache.hadoop.hbase.util.Bytes;
60 import org.apache.hadoop.hbase.util.Pair;
61 import org.apache.hadoop.io.compress.GzipCodec;
62 import org.apache.hadoop.net.NetUtils;
63 import org.apache.hadoop.util.StringUtils;
64 import org.junit.Test;
65 import org.junit.experimental.categories.Category;
66 import org.mockito.Mockito;
67 import org.mockito.invocation.InvocationOnMock;
68 import org.mockito.stubbing.Answer;
69
70 import com.google.protobuf.BlockingService;
71 import com.google.protobuf.Descriptors.MethodDescriptor;
72 import com.google.protobuf.Message;
73 import com.google.protobuf.RpcController;
74 import com.google.protobuf.ServiceException;
75
76
77
78
79 @Category(SmallTests.class)
80 public class TestIPC {
81 public static final Log LOG = LogFactory.getLog(TestIPC.class);
82 static byte [] CELL_BYTES = Bytes.toBytes("xyz");
83 static Cell CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES);
84
85
86
87
88 private static final BlockingService SERVICE =
89 TestRpcServiceProtos.TestProtobufRpcProto.newReflectiveBlockingService(
90 new TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface() {
91
92 @Override
93 public EmptyResponseProto ping(RpcController controller,
94 EmptyRequestProto request) throws ServiceException {
95
96 return null;
97 }
98
99 @Override
100 public EmptyResponseProto error(RpcController controller,
101 EmptyRequestProto request) throws ServiceException {
102
103 return null;
104 }
105
106 @Override
107 public EchoResponseProto echo(RpcController controller, EchoRequestProto request)
108 throws ServiceException {
109 if (controller instanceof PayloadCarryingRpcController) {
110 PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController)controller;
111
112
113
114 CellScanner cellScanner = pcrc.cellScanner();
115 List<Cell> list = new ArrayList<Cell>();
116 try {
117 while(cellScanner.advance()) {
118 list.add(cellScanner.current());
119 }
120 } catch (IOException e) {
121 throw new ServiceException(e);
122 }
123 cellScanner = CellUtil.createCellScanner(list);
124 ((PayloadCarryingRpcController)controller).setCellScanner(cellScanner);
125 }
126 return EchoResponseProto.newBuilder().setMessage(request.getMessage()).build();
127 }
128 });
129
130
131
132
133
134 private static class TestRpcServer extends RpcServer {
135 TestRpcServer() throws IOException {
136 super(null, "testRpcServer",
137 Lists.newArrayList(new BlockingServiceAndInterface(SERVICE, null)),
138 new InetSocketAddress("0.0.0.0", 0), 1, 1,
139 HBaseConfiguration.create(), 0);
140 }
141
142 @Override
143 public Pair<Message, CellScanner> call(BlockingService service,
144 MethodDescriptor md, Message param, CellScanner cellScanner,
145 long receiveTime, MonitoredRPCHandler status) throws IOException {
146 return super.call(service, md, param, cellScanner, receiveTime, status);
147 }
148 }
149
150
151
152
153
154
155
156
157
158
159 @Test
160 public void testCompressCellBlock()
161 throws IOException, InterruptedException, SecurityException, NoSuchMethodException {
162
163 Configuration conf = HBaseConfiguration.create();
164 conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName());
165 TestRpcServer rpcServer = new TestRpcServer();
166 RpcClient client = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT);
167 List<Cell> cells = new ArrayList<Cell>();
168 int count = 3;
169 for (int i = 0; i < count; i++) cells.add(CELL);
170 try {
171 rpcServer.start();
172 InetSocketAddress address = rpcServer.getListenerAddress();
173 MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo");
174 EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build();
175 Pair<Message, CellScanner> r = client.call(md, param, CellUtil.createCellScanner(cells),
176 md.getOutputType().toProto(), User.getCurrent(), address, 0);
177 int index = 0;
178 while (r.getSecond().advance()) {
179 assertTrue(CELL.equals(r.getSecond().current()));
180 index++;
181 }
182 assertEquals(count, index);
183 } finally {
184 client.stop();
185 rpcServer.stop();
186 }
187 }
188
189 @Test
190 public void testRTEDuringConnectionSetup() throws Exception {
191 Configuration conf = HBaseConfiguration.create();
192 SocketFactory spyFactory = spy(NetUtils.getDefaultSocketFactory(conf));
193 Mockito.doAnswer(new Answer<Socket>() {
194 @Override
195 public Socket answer(InvocationOnMock invocation) throws Throwable {
196 Socket s = spy((Socket)invocation.callRealMethod());
197 doThrow(new RuntimeException("Injected fault")).when(s).setSoTimeout(anyInt());
198 return s;
199 }
200 }).when(spyFactory).createSocket();
201
202 TestRpcServer rpcServer = new TestRpcServer();
203 RpcClient client = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory);
204 try {
205 rpcServer.start();
206 InetSocketAddress address = rpcServer.getListenerAddress();
207 MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo");
208 EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build();
209 client.call(md, param, null, null, User.getCurrent(), address, 0);
210 fail("Expected an exception to have been thrown!");
211 } catch (Exception e) {
212 LOG.info("Caught expected exception: " + e.toString());
213 assertTrue(StringUtils.stringifyException(e).contains("Injected fault"));
214 } finally {
215 client.stop();
216 rpcServer.stop();
217 }
218 }
219
220 public static void main(String[] args)
221 throws IOException, SecurityException, NoSuchMethodException, InterruptedException {
222 if (args.length != 2) {
223 System.out.println("Usage: TestIPC <CYCLES> <CELLS_PER_CYCLE>");
224 return;
225 }
226
227
228 int cycles = Integer.parseInt(args[0]);
229 int cellcount = Integer.parseInt(args[1]);
230 Configuration conf = HBaseConfiguration.create();
231 TestRpcServer rpcServer = new TestRpcServer();
232 RpcClient client = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT);
233 KeyValue kv = KeyValueUtil.ensureKeyValue(CELL);
234 Put p = new Put(kv.getRow());
235 for (int i = 0; i < cellcount; i++) {
236 p.add(kv);
237 }
238 RowMutations rm = new RowMutations(kv.getRow());
239 rm.add(p);
240 try {
241 rpcServer.start();
242 InetSocketAddress address = rpcServer.getListenerAddress();
243 long startTime = System.currentTimeMillis();
244 User user = User.getCurrent();
245 for (int i = 0; i < cycles; i++) {
246 List<CellScannable> cells = new ArrayList<CellScannable>();
247
248 Message param = RequestConverter.buildNoDataMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm, cells);
249 CellScanner cellScanner = CellUtil.createCellScanner(cells);
250 if (i % 1000 == 0) {
251 LOG.info("" + i);
252
253
254
255 }
256 Pair<Message, CellScanner> response =
257 client.call(null, param, cellScanner, null, user, address, 0);
258
259
260
261
262
263
264 }
265 LOG.info("Cycled " + cycles + " time(s) with " + cellcount + " cell(s) in " +
266 (System.currentTimeMillis() - startTime) + "ms");
267 } finally {
268 client.stop();
269 rpcServer.stop();
270 }
271 }
272 }