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 java.io.IOException;
21
22 import org.apache.hadoop.conf.Configuration;
23 import org.apache.hadoop.hbase.RegionLocations;
24 import org.apache.hadoop.hbase.TableName;
25 import org.apache.hadoop.hbase.HRegionInfo;
26 import org.apache.hadoop.hbase.HRegionLocation;
27 import org.apache.hadoop.hbase.ServerName;
28 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
29 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
30 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
31 import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
32 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
33 import org.mockito.Mockito;
34 import org.mockito.invocation.InvocationOnMock;
35 import org.mockito.stubbing.Answer;
36
37
38
39
40 public class HConnectionTestingUtility {
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57 public static ClusterConnection getMockedConnection(final Configuration conf)
58 throws ZooKeeperConnectionException {
59 HConnectionKey connectionKey = new HConnectionKey(conf);
60 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
61 HConnectionImplementation connection =
62 ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
63 if (connection == null) {
64 connection = Mockito.mock(HConnectionImplementation.class);
65 Mockito.when(connection.getConfiguration()).thenReturn(conf);
66 Mockito.when(connection.getRpcControllerFactory()).thenReturn(
67 Mockito.mock(RpcControllerFactory.class));
68
69 RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
70 Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
71 ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
72 }
73 return connection;
74 }
75 }
76
77
78
79
80 private static void mockRegionLocator(final HConnectionImplementation connection) {
81 try {
82 Mockito.when(connection.getRegionLocator(Mockito.any(TableName.class))).thenAnswer(
83 new Answer<RegionLocator>() {
84 @Override
85 public RegionLocator answer(InvocationOnMock invocation) throws Throwable {
86 TableName tableName = (TableName) invocation.getArguments()[0];
87 return new HRegionLocator(tableName, connection);
88 }
89 });
90 } catch (IOException e) {
91 }
92 }
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123 public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf,
124 final AdminProtos.AdminService.BlockingInterface admin,
125 final ClientProtos.ClientService.BlockingInterface client,
126 final ServerName sn, final HRegionInfo hri)
127 throws IOException {
128 HConnectionImplementation c = Mockito.mock(HConnectionImplementation.class);
129 Mockito.when(c.getConfiguration()).thenReturn(conf);
130 ConnectionManager.CONNECTION_INSTANCES.put(new HConnectionKey(conf), c);
131 Mockito.doNothing().when(c).close();
132
133 final HRegionLocation loc = new HRegionLocation(hri, sn);
134 mockRegionLocator(c);
135 Mockito.when(c.getRegionLocation((TableName) Mockito.any(),
136 (byte[]) Mockito.any(), Mockito.anyBoolean())).
137 thenReturn(loc);
138 Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any())).
139 thenReturn(loc);
140 Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any(),
141 Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt()))
142 .thenReturn(new RegionLocations(loc));
143 if (admin != null) {
144
145 Mockito.when(c.getAdmin(Mockito.any(ServerName.class))).
146 thenReturn(admin);
147 }
148 if (client != null) {
149
150 Mockito.when(c.getClient(Mockito.any(ServerName.class))).
151 thenReturn(client);
152 }
153 NonceGenerator ng = Mockito.mock(NonceGenerator.class);
154 Mockito.when(c.getNonceGenerator()).thenReturn(ng);
155 Mockito.when(c.getAsyncProcess()).thenReturn(
156 new AsyncProcess(c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false,
157 RpcControllerFactory.instantiate(conf)));
158 Mockito.doNothing().when(c).incCount();
159 Mockito.doNothing().when(c).decCount();
160 Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn(
161 RpcRetryingCallerFactory.instantiate(conf,
162 RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null));
163 Mockito.when(c.getRpcControllerFactory()).thenReturn(Mockito.mock(RpcControllerFactory.class));
164 HTableInterface t = Mockito.mock(HTableInterface.class);
165 Mockito.when(c.getTable((TableName)Mockito.any())).thenReturn(t);
166 ResultScanner rs = Mockito.mock(ResultScanner.class);
167 Mockito.when(t.getScanner((Scan)Mockito.any())).thenReturn(rs);
168 return c;
169 }
170
171
172
173
174
175
176
177
178
179
180
181
182
183 public static ClusterConnection getSpiedConnection(final Configuration conf)
184 throws IOException {
185 HConnectionKey connectionKey = new HConnectionKey(conf);
186 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
187 HConnectionImplementation connection =
188 ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
189 if (connection == null) {
190 connection = Mockito.spy(new HConnectionImplementation(conf, true));
191 ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
192 }
193 return connection;
194 }
195 }
196
197 public static ClusterConnection getSpiedClusterConnection(final Configuration conf)
198 throws IOException {
199 HConnectionKey connectionKey = new HConnectionKey(conf);
200 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
201 HConnectionImplementation connection =
202 ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
203 if (connection == null) {
204 connection = Mockito.spy(new HConnectionImplementation(conf, true));
205 ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
206 }
207 return connection;
208 }
209 }
210
211
212
213
214 public static int getConnectionCount() {
215 synchronized (ConnectionManager.CONNECTION_INSTANCES) {
216 return ConnectionManager.CONNECTION_INSTANCES.size();
217 }
218 }
219 }