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.hamcrest.CoreMatchers.instanceOf;
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertFalse;
23 import static org.junit.Assert.assertNotNull;
24 import static org.junit.Assert.assertThat;
25 import static org.junit.Assert.assertTrue;
26 import static org.junit.Assert.fail;
27 import static org.mockito.Matchers.any;
28 import static org.mockito.Matchers.anyBoolean;
29 import static org.mockito.Matchers.anyInt;
30 import static org.mockito.Mockito.when;
31
32 import java.io.IOException;
33 import java.util.Iterator;
34 import java.util.concurrent.ExecutorService;
35 import java.util.concurrent.Executors;
36
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.hbase.Cell;
39 import org.apache.hadoop.hbase.CellScanner;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.KeyValue.Type;
42 import org.apache.hadoop.hbase.RegionLocations;
43 import org.apache.hadoop.hbase.TableName;
44 import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
45 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
46 import org.apache.hadoop.hbase.testclassification.SmallTests;
47 import org.apache.hadoop.hbase.util.Bytes;
48 import org.junit.After;
49 import org.junit.Before;
50 import org.junit.Test;
51 import org.junit.experimental.categories.Category;
52 import org.mockito.InOrder;
53 import org.mockito.Mockito;
54 import org.mockito.invocation.InvocationOnMock;
55 import org.mockito.stubbing.Answer;
56
57
58
59
60 @Category(SmallTests.class)
61 public class TestClientScanner {
62
63 Scan scan;
64 ExecutorService pool;
65 Configuration conf;
66
67 ClusterConnection clusterConn;
68 RpcRetryingCallerFactory rpcFactory;
69 RpcControllerFactory controllerFactory;
70
71 @Before
72 @SuppressWarnings("deprecation")
73 public void setup() throws IOException {
74 clusterConn = Mockito.mock(ClusterConnection.class);
75 rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);
76 controllerFactory = Mockito.mock(RpcControllerFactory.class);
77 pool = Executors.newSingleThreadExecutor();
78 scan = new Scan();
79 conf = new Configuration();
80 Mockito.when(clusterConn.getConfiguration()).thenReturn(conf);
81 }
82
83 @After
84 public void teardown() {
85 if (null != pool) {
86 pool.shutdownNow();
87 }
88 }
89
90 private static class MockClientScanner extends ClientSimpleScanner {
91
92 private boolean rpcFinished = false;
93 private boolean rpcFinishedFired = false;
94 private boolean initialized = false;
95
96 public MockClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
97 ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
98 RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
99 throws IOException {
100 super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool,
101 primaryOperationTimeout);
102 }
103
104 @Override
105 protected boolean moveToNextRegion() {
106 if (!initialized) {
107 initialized = true;
108 return super.moveToNextRegion();
109 }
110 if (!rpcFinished) {
111 return super.moveToNextRegion();
112 }
113
114 if (rpcFinishedFired) {
115 throw new RuntimeException("Expected nextScanner to only be called once after " +
116 " short-circuit was triggered.");
117 }
118 rpcFinishedFired = true;
119 return false;
120 }
121
122 public void setRpcFinished(boolean rpcFinished) {
123 this.rpcFinished = rpcFinished;
124 }
125 }
126
127 @Test
128 @SuppressWarnings("unchecked")
129 public void testNoResultsHint() throws IOException {
130 final Result[] results = new Result[1];
131 KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
132 Type.Maximum);
133 results[0] = Result.create(new Cell[] {kv1});
134
135 RpcRetryingCaller<Result[]> caller = Mockito.mock(RpcRetryingCaller.class);
136
137 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
138 Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class),
139 Mockito.anyInt())).thenAnswer(new Answer<Result[]>() {
140 private int count = 0;
141 @Override
142 public Result[] answer(InvocationOnMock invocation) throws Throwable {
143 ScannerCallableWithReplicas callable = invocation.getArgumentAt(0,
144 ScannerCallableWithReplicas.class);
145 switch (count) {
146 case 0:
147 count++;
148 callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.UNKNOWN);
149 return results;
150 case 1:
151 case 2:
152 count++;
153 return new Result[0];
154 default:
155 throw new RuntimeException("Expected only 2 invocations");
156 }
157 }
158 });
159
160
161 scan.setCaching(100);
162 scan.setMaxResultSize(1000*1000);
163
164 try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
165 clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
166
167 scanner.setRpcFinished(true);
168
169 InOrder inOrder = Mockito.inOrder(caller);
170
171 scanner.loadCache();
172
173
174
175 inOrder.verify(caller, Mockito.times(2)).callWithoutRetries(
176 Mockito.any(RetryingCallable.class), Mockito.anyInt());
177
178 assertEquals(1, scanner.cache.size());
179 Result r = scanner.cache.poll();
180 assertNotNull(r);
181 CellScanner cs = r.cellScanner();
182 assertTrue(cs.advance());
183 assertEquals(kv1, cs.current());
184 assertFalse(cs.advance());
185 }
186 }
187
188 @Test
189 @SuppressWarnings("unchecked")
190 public void testSizeLimit() throws IOException {
191 final Result[] results = new Result[1];
192 KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
193 Type.Maximum);
194 results[0] = Result.create(new Cell[] {kv1});
195
196 RpcRetryingCaller<Result[]> caller = Mockito.mock(RpcRetryingCaller.class);
197
198 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
199 Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class),
200 Mockito.anyInt())).thenAnswer(new Answer<Result[]>() {
201 private int count = 0;
202 @Override
203 public Result[] answer(InvocationOnMock invocation) throws Throwable {
204 ScannerCallableWithReplicas callable = invocation.getArgumentAt(0,
205 ScannerCallableWithReplicas.class);
206 switch (count) {
207 case 0:
208 count++;
209
210 callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES);
211 return results;
212 case 1:
213 count++;
214 return null;
215 default:
216 throw new RuntimeException("Expected only 2 invocations");
217 }
218 }
219 });
220
221 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
222
223
224 scan.setCaching(100);
225
226 scan.setMaxResultSize(1);
227
228 try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
229 clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
230 InOrder inOrder = Mockito.inOrder(caller);
231
232 scanner.loadCache();
233
234 inOrder.verify(caller, Mockito.times(1)).callWithoutRetries(
235 Mockito.any(RetryingCallable.class), Mockito.anyInt());
236
237 assertEquals(1, scanner.cache.size());
238 Result r = scanner.cache.poll();
239 assertNotNull(r);
240 CellScanner cs = r.cellScanner();
241 assertTrue(cs.advance());
242 assertEquals(kv1, cs.current());
243 assertFalse(cs.advance());
244 }
245 }
246
247 @Test
248 @SuppressWarnings("unchecked")
249 public void testCacheLimit() throws IOException {
250 KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
251 Type.Maximum);
252 KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
253 Type.Maximum);
254 KeyValue kv3 = new KeyValue(Bytes.toBytes("row3"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
255 Type.Maximum);
256 final Result[] results = new Result[] {Result.create(new Cell[] {kv1}),
257 Result.create(new Cell[] {kv2}), Result.create(new Cell[] {kv3})};
258
259 RpcRetryingCaller<Result[]> caller = Mockito.mock(RpcRetryingCaller.class);
260
261 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
262 Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class),
263 Mockito.anyInt())).thenAnswer(new Answer<Result[]>() {
264 private int count = 0;
265 @Override
266 public Result[] answer(InvocationOnMock invocation) throws Throwable {
267 ScannerCallableWithReplicas callable = invocation.getArgumentAt(0,
268 ScannerCallableWithReplicas.class);
269 switch (count) {
270 case 0:
271 count++;
272
273 callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES);
274 return results;
275 case 1:
276 count++;
277 return null;
278 default:
279 throw new RuntimeException("Expected only 2 invocations");
280 }
281 }
282 });
283
284 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
285
286
287 scan.setCaching(1);
288
289 scan.setMaxResultSize(1000*1000);
290
291 try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
292 clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
293 InOrder inOrder = Mockito.inOrder(caller);
294
295 scanner.loadCache();
296
297 inOrder.verify(caller, Mockito.times(1)).callWithoutRetries(
298 Mockito.any(RetryingCallable.class), Mockito.anyInt());
299
300 assertEquals(3, scanner.cache.size());
301 Result r = scanner.cache.poll();
302 assertNotNull(r);
303 CellScanner cs = r.cellScanner();
304 assertTrue(cs.advance());
305 assertEquals(kv1, cs.current());
306 assertFalse(cs.advance());
307
308 r = scanner.cache.poll();
309 assertNotNull(r);
310 cs = r.cellScanner();
311 assertTrue(cs.advance());
312 assertEquals(kv2, cs.current());
313 assertFalse(cs.advance());
314
315 r = scanner.cache.poll();
316 assertNotNull(r);
317 cs = r.cellScanner();
318 assertTrue(cs.advance());
319 assertEquals(kv3, cs.current());
320 assertFalse(cs.advance());
321 }
322 }
323
324 @Test
325 @SuppressWarnings("unchecked")
326 public void testNoMoreResults() throws IOException {
327 final Result[] results = new Result[1];
328 KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
329 Type.Maximum);
330 results[0] = Result.create(new Cell[] {kv1});
331
332 RpcRetryingCaller<Result[]> caller = Mockito.mock(RpcRetryingCaller.class);
333
334 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
335 Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class),
336 Mockito.anyInt())).thenAnswer(new Answer<Result[]>() {
337 private int count = 0;
338 @Override
339 public Result[] answer(InvocationOnMock invocation) throws Throwable {
340 ScannerCallableWithReplicas callable = invocation.getArgumentAt(0,
341 ScannerCallableWithReplicas.class);
342 switch (count) {
343 case 0:
344 count++;
345 callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO);
346 return results;
347 case 1:
348 count++;
349 return null;
350 default:
351 throw new RuntimeException("Expected only 2 invocations");
352 }
353 }
354 });
355
356 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
357
358
359 scan.setCaching(100);
360 scan.setMaxResultSize(1000*1000);
361
362 try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
363 clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
364 scanner.setRpcFinished(true);
365
366 InOrder inOrder = Mockito.inOrder(caller);
367
368 scanner.loadCache();
369
370 inOrder.verify(caller, Mockito.times(1)).callWithoutRetries(
371 Mockito.any(RetryingCallable.class), Mockito.anyInt());
372
373 assertEquals(1, scanner.cache.size());
374 Result r = scanner.cache.poll();
375 assertNotNull(r);
376 CellScanner cs = r.cellScanner();
377 assertTrue(cs.advance());
378 assertEquals(kv1, cs.current());
379 assertFalse(cs.advance());
380 }
381 }
382
383 @Test
384 @SuppressWarnings("unchecked")
385 public void testMoreResults() throws IOException {
386 final Result[] results1 = new Result[1];
387 KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
388 Type.Maximum);
389 results1[0] = Result.create(new Cell[] {kv1});
390
391 final Result[] results2 = new Result[1];
392 KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
393 Type.Maximum);
394 results2[0] = Result.create(new Cell[] {kv2});
395
396
397 RpcRetryingCaller<Result[]> caller = Mockito.mock(RpcRetryingCaller.class);
398
399 Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
400 Mockito.when(caller.callWithoutRetries(Mockito.any(RetryingCallable.class),
401 Mockito.anyInt())).thenAnswer(new Answer<Result[]>() {
402 private int count = 0;
403 @Override
404 public Result[] answer(InvocationOnMock invocation) throws Throwable {
405 ScannerCallableWithReplicas callable = invocation.getArgumentAt(0,
406 ScannerCallableWithReplicas.class);
407 switch (count) {
408 case 0:
409 count++;
410 callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES);
411 return results1;
412 case 1:
413 count++;
414
415 callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO);
416 return results2;
417 case 2:
418 count++;
419 return null;
420 default:
421 throw new RuntimeException("Expected only 3 invocations");
422 }
423 }
424 });
425
426
427 scan.setCaching(100);
428 scan.setMaxResultSize(1000*1000);
429
430 try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
431 clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
432 InOrder inOrder = Mockito.inOrder(caller);
433 scanner.setRpcFinished(true);
434
435 scanner.loadCache();
436
437 inOrder.verify(caller, Mockito.times(2)).callWithoutRetries(
438 Mockito.any(RetryingCallable.class), Mockito.anyInt());
439
440 assertEquals(2, scanner.cache.size());
441 Result r = scanner.cache.poll();
442 assertNotNull(r);
443 CellScanner cs = r.cellScanner();
444 assertTrue(cs.advance());
445 assertEquals(kv1, cs.current());
446 assertFalse(cs.advance());
447
448 r = scanner.cache.poll();
449 assertNotNull(r);
450 cs = r.cellScanner();
451 assertTrue(cs.advance());
452 assertEquals(kv2, cs.current());
453 assertFalse(cs.advance());
454 }
455 }
456
457
458
459
460
461 @Test (timeout = 30000)
462 public void testExceptionsFromReplicasArePropagated() throws IOException {
463 scan.setConsistency(Consistency.TIMELINE);
464
465
466
467 rpcFactory = new MockRpcRetryingCallerFactory(conf);
468 conf.set(RpcRetryingCallerFactory.CUSTOM_CALLER_CONF_KEY,
469 MockRpcRetryingCallerFactory.class.getName());
470
471
472 when(clusterConn.locateRegion((TableName)any(), (byte[])any(), anyBoolean(),
473 anyBoolean(), anyInt())).thenReturn(new RegionLocations(null, null, null));
474
475 try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
476 clusterConn, rpcFactory, new RpcControllerFactory(conf), pool, Integer.MAX_VALUE)) {
477 Iterator<Result> iter = scanner.iterator();
478 while (iter.hasNext()) {
479 iter.next();
480 }
481 fail("Should have failed with RetriesExhaustedException");
482 } catch (RuntimeException expected) {
483 assertThat(expected.getCause(), instanceOf(RetriesExhaustedException.class));
484 }
485 }
486
487 public static class MockRpcRetryingCallerFactory extends RpcRetryingCallerFactory {
488
489 public MockRpcRetryingCallerFactory(Configuration conf) {
490 super(conf);
491 }
492
493 @Override
494 public <T> RpcRetryingCaller<T> newCaller() {
495 return new RpcRetryingCaller<T>(0, 0, 0, 0) {
496 @Override
497 public void cancel() {
498 }
499 @Override
500 public T callWithRetries(RetryingCallable<T> callable, int callTimeout)
501 throws IOException, RuntimeException {
502 throw new IOException("Scanner exception");
503 }
504
505 @Override
506 public T callWithoutRetries(RetryingCallable<T> callable, int callTimeout)
507 throws IOException, RuntimeException {
508 try {
509 return callable.call(callTimeout);
510 } catch (IOException e) {
511 throw e;
512 } catch (Exception e) {
513 throw new RuntimeException(e);
514 }
515 }
516 };
517 }
518 }
519 }