1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.client;
19
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertNotNull;
22
23 import java.io.IOException;
24 import java.util.concurrent.atomic.AtomicLong;
25
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.RegionLocations;
28 import org.apache.hadoop.hbase.TableName;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.HRegionInfo;
31 import org.apache.hadoop.hbase.HRegionLocation;
32 import org.apache.hadoop.hbase.ServerName;
33 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
34 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
35 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
36 import org.apache.hadoop.hbase.util.Threads;
37 import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
38 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
39 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
40 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
41 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
42 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
43 import org.mockito.Mockito;
44 import org.mockito.invocation.InvocationOnMock;
45 import org.mockito.stubbing.Answer;
46
47
48
49
50 public class HConnectionTestingUtility {
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67 public static ClusterConnection getMockedConnection(final Configuration conf)
68 throws ZooKeeperConnectionException {
69 HConnectionKey connectionKey = new HConnectionKey(conf);
70 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
71 HConnectionImplementation connection =
72 ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
73 if (connection == null) {
74 connection = Mockito.mock(HConnectionImplementation.class);
75 Mockito.when(connection.getConfiguration()).thenReturn(conf);
76 Mockito.when(connection.getRpcControllerFactory()).thenReturn(
77 Mockito.mock(RpcControllerFactory.class));
78
79 RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
80 Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
81 ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
82 }
83 return connection;
84 }
85 }
86
87
88
89
90 private static void mockRegionLocator(final HConnectionImplementation connection) {
91 try {
92 Mockito.when(connection.getRegionLocator(Mockito.any(TableName.class))).thenAnswer(
93 new Answer<RegionLocator>() {
94 @Override
95 public RegionLocator answer(InvocationOnMock invocation) throws Throwable {
96 TableName tableName = (TableName) invocation.getArguments()[0];
97 return new HRegionLocator(tableName, connection);
98 }
99 });
100 } catch (IOException e) {
101 }
102 }
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133 public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf,
134 final AdminProtos.AdminService.BlockingInterface admin,
135 final ClientProtos.ClientService.BlockingInterface client,
136 final ServerName sn, final HRegionInfo hri)
137 throws IOException {
138 HConnectionImplementation c = Mockito.mock(HConnectionImplementation.class);
139 Mockito.when(c.getConfiguration()).thenReturn(conf);
140 ConnectionManager.CONNECTION_INSTANCES.put(new HConnectionKey(conf), c);
141 Mockito.doNothing().when(c).close();
142
143 final HRegionLocation loc = new HRegionLocation(hri, sn);
144 mockRegionLocator(c);
145 Mockito.when(c.getRegionLocation((TableName) Mockito.any(),
146 (byte[]) Mockito.any(), Mockito.anyBoolean())).
147 thenReturn(loc);
148 Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any())).
149 thenReturn(loc);
150 Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any(),
151 Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt()))
152 .thenReturn(new RegionLocations(loc));
153 if (admin != null) {
154
155 Mockito.when(c.getAdmin(Mockito.any(ServerName.class))).
156 thenReturn(admin);
157 }
158 if (client != null) {
159
160 Mockito.when(c.getClient(Mockito.any(ServerName.class))).
161 thenReturn(client);
162 }
163 NonceGenerator ng = Mockito.mock(NonceGenerator.class);
164 Mockito.when(c.getNonceGenerator()).thenReturn(ng);
165 AsyncProcess asyncProcess =
166 new AsyncProcess(c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false,
167 RpcControllerFactory.instantiate(conf), conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
168 HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
169 Mockito.when(c.getAsyncProcess()).thenReturn(asyncProcess);
170 Mockito.doNothing().when(c).incCount();
171 Mockito.doNothing().when(c).decCount();
172 Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn(
173 RpcRetryingCallerFactory.instantiate(conf,
174 RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null));
175 Mockito.when(c.getRpcControllerFactory()).thenReturn(Mockito.mock(RpcControllerFactory.class));
176 HTableInterface t = Mockito.mock(HTableInterface.class);
177 Mockito.when(c.getTable((TableName)Mockito.any())).thenReturn(t);
178 ResultScanner rs = Mockito.mock(ResultScanner.class);
179 Mockito.when(t.getScanner((Scan)Mockito.any())).thenReturn(rs);
180 return c;
181 }
182
183
184
185
186
187
188
189
190
191
192
193
194
195 public static ClusterConnection getSpiedConnection(final Configuration conf)
196 throws IOException {
197 HConnectionKey connectionKey = new HConnectionKey(conf);
198 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
199 HConnectionImplementation connection =
200 ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
201 if (connection == null) {
202 connection = Mockito.spy(new HConnectionImplementation(conf, true));
203 ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
204 }
205 return connection;
206 }
207 }
208
209 public static ClusterConnection getSpiedClusterConnection(final Configuration conf)
210 throws IOException {
211 HConnectionKey connectionKey = new HConnectionKey(conf);
212 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
213 HConnectionImplementation connection =
214 ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
215 if (connection == null) {
216 connection = Mockito.spy(new HConnectionImplementation(conf, true));
217 ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
218 }
219 return connection;
220 }
221 }
222
223
224
225
226 public static int getConnectionCount() {
227 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
228 return ConnectionManager.CONNECTION_INSTANCES.size();
229 }
230 }
231
232 public static HConnectionImplementation requireHConnImpl(Connection conn) {
233 assertNotNull("Cannot operate on a null Connection", conn);
234 assertEquals("This method requires an HConnectionImplementation",
235 HConnectionImplementation.class, conn.getClass());
236 return (HConnectionImplementation) conn;
237 }
238
239 public static RecoverableZooKeeper unwrapZK(Connection conn) throws IOException {
240 return requireHConnImpl(conn).getKeepAliveZooKeeperWatcher().getRecoverableZooKeeper();
241 }
242
243 public static void clearRegionCache(Connection conn) throws IOException {
244 requireHConnImpl(conn).clearRegionCache();
245 }
246
247
248
249
250 public static class SleepAtFirstRpcCall extends BaseRegionObserver {
251 static final AtomicLong ct = new AtomicLong(0);
252 static final String SLEEP_TIME_CONF_KEY =
253 "hbase.coprocessor.SleepAtFirstRpcCall.sleepTime";
254 static final long DEFAULT_SLEEP_TIME = 2000;
255 static final AtomicLong sleepTime = new AtomicLong(DEFAULT_SLEEP_TIME);
256
257 public SleepAtFirstRpcCall() {
258 }
259
260 @Override
261 public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
262 RegionCoprocessorEnvironment env = e.getEnvironment();
263 Configuration conf = env.getConfiguration();
264 sleepTime.set(conf.getLong(SLEEP_TIME_CONF_KEY, DEFAULT_SLEEP_TIME));
265 }
266
267 @Override
268 public Result postIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
269 final Increment increment, final Result result) throws IOException {
270 if (ct.incrementAndGet() == 1) {
271 Threads.sleep(sleepTime.get());
272 }
273 return result;
274 }
275
276 @Override
277 public Result postAppend(final ObserverContext<RegionCoprocessorEnvironment> e,
278 final Append append, final Result result) throws IOException {
279 if (ct.incrementAndGet() == 1) {
280 Threads.sleep(sleepTime.get());
281 }
282 return result;
283 }
284 }
285
286 }