1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.client;
21
22
23 import org.apache.hadoop.conf.Configuration;
24 import org.apache.hadoop.hbase.TableName;
25 import org.apache.hadoop.hbase.HConstants;
26 import org.apache.hadoop.hbase.HRegionInfo;
27 import org.apache.hadoop.hbase.HRegionLocation;
28 import org.apache.hadoop.hbase.testclassification.MediumTests;
29 import org.apache.hadoop.hbase.ServerName;
30 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
31 import org.apache.hadoop.hbase.util.Bytes;
32 import org.apache.hadoop.hbase.util.Threads;
33 import org.junit.Assert;
34 import org.junit.Test;
35 import org.junit.experimental.categories.Category;
36 import org.mockito.Mockito;
37
38 import java.io.IOException;
39 import java.io.InterruptedIOException;
40 import java.util.ArrayList;
41 import java.util.Arrays;
42 import java.util.List;
43 import java.util.Map;
44 import java.util.concurrent.BlockingQueue;
45 import java.util.concurrent.ExecutorService;
46 import java.util.concurrent.Future;
47 import java.util.concurrent.LinkedBlockingQueue;
48 import java.util.concurrent.SynchronousQueue;
49 import java.util.concurrent.ThreadFactory;
50 import java.util.concurrent.ThreadPoolExecutor;
51 import java.util.concurrent.TimeUnit;
52 import java.util.concurrent.atomic.AtomicBoolean;
53 import java.util.concurrent.atomic.AtomicInteger;
54
55 @Category(MediumTests.class)
56 public class TestAsyncProcess {
57 private static final TableName DUMMY_TABLE =
58 TableName.valueOf("DUMMY_TABLE");
59 private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();
60 private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes();
61 private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes();
62 private static final byte[] FAILS = "FAILS".getBytes();
63 private static final Configuration conf = new Configuration();
64
65 private static ServerName sn = ServerName.valueOf("localhost:10,1254");
66 private static ServerName sn2 = ServerName.valueOf("localhost:140,12540");
67 private static HRegionInfo hri1 =
68 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1);
69 private static HRegionInfo hri2 =
70 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_2, HConstants.EMPTY_END_ROW, false, 2);
71 private static HRegionInfo hri3 =
72 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_3, HConstants.EMPTY_END_ROW, false, 3);
73 private static HRegionLocation loc1 = new HRegionLocation(hri1, sn);
74 private static HRegionLocation loc2 = new HRegionLocation(hri2, sn);
75 private static HRegionLocation loc3 = new HRegionLocation(hri3, sn2);
76
77 private static final String success = "success";
78 private static Exception failure = new Exception("failure");
79
80 static class CountingThreadFactory implements ThreadFactory {
81 final AtomicInteger nbThreads;
82 ThreadFactory realFactory = Threads.newDaemonThreadFactory("test-TestAsyncProcess");
83 @Override
84 public Thread newThread(Runnable r) {
85 nbThreads.incrementAndGet();
86 return realFactory.newThread(r);
87 }
88
89 CountingThreadFactory(AtomicInteger nbThreads){
90 this.nbThreads = nbThreads;
91 }
92 }
93
94 static class MyAsyncProcess<Res> extends AsyncProcess<Res> {
95 final AtomicInteger nbMultiResponse = new AtomicInteger();
96 final AtomicInteger nbActions = new AtomicInteger();
97
98 public MyAsyncProcess(HConnection hc, AsyncProcessCallback<Res> callback, Configuration conf) {
99 this(hc, callback, conf, new AtomicInteger());
100 }
101
102 public MyAsyncProcess(HConnection hc, AsyncProcessCallback<Res> callback, Configuration conf,
103 AtomicInteger nbThreads) {
104 super(hc, DUMMY_TABLE, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
105 new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
106 callback, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
107 }
108
109 @Override
110 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
111 final MultiResponse mr = createMultiResponse(callable.getLocation(), callable.getMulti(),
112 nbMultiResponse, nbActions);
113 return new RpcRetryingCaller<MultiResponse>(100, 10, 9) {
114 @Override
115 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable, int to)
116 throws IOException, RuntimeException {
117 try {
118
119
120 Thread.sleep(1000);
121 } catch (InterruptedException e) {
122
123 }
124 return mr;
125 }
126 };
127 }
128 }
129
130 static class CallerWithFailure extends RpcRetryingCaller<MultiResponse>{
131
132 public CallerWithFailure() {
133 super(100, 100, 9);
134 }
135
136 @Override
137 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable, int to)
138 throws IOException, RuntimeException {
139 throw new IOException("test");
140 }
141 }
142
143 static class AsyncProcessWithFailure<Res> extends MyAsyncProcess<Res> {
144
145 public AsyncProcessWithFailure(HConnection hc, Configuration conf) {
146 super(hc, null, conf, new AtomicInteger());
147 serverTrackerTimeout = 1;
148 }
149
150 @Override
151 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
152 return new CallerWithFailure();
153 }
154 }
155
156
157 static MultiResponse createMultiResponse(final HRegionLocation loc,
158 final MultiAction<Row> multi, AtomicInteger nbMultiResponse, AtomicInteger nbActions) {
159 final MultiResponse mr = new MultiResponse();
160 nbMultiResponse.incrementAndGet();
161 for (Map.Entry<byte[], List<Action<Row>>> entry : multi.actions.entrySet()) {
162 for (Action a : entry.getValue()) {
163 nbActions.incrementAndGet();
164 if (Arrays.equals(FAILS, a.getAction().getRow())) {
165 mr.add(loc.getRegionInfo().getRegionName(), a.getOriginalIndex(), failure);
166 } else {
167 mr.add(loc.getRegionInfo().getRegionName(), a.getOriginalIndex(), success);
168 }
169 }
170 }
171 return mr;
172 }
173
174
175
176 static class MyConnectionImpl extends HConnectionManager.HConnectionImplementation {
177 MyAsyncProcess<?> ap;
178 final AtomicInteger nbThreads = new AtomicInteger(0);
179 final static Configuration c = new Configuration();
180
181 static {
182 c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
183 }
184
185 protected MyConnectionImpl() {
186 super(c);
187 }
188
189 protected MyConnectionImpl(Configuration conf) {
190 super(conf);
191 }
192
193 @Override
194 protected <R> AsyncProcess createAsyncProcess(TableName tableName,
195 ExecutorService pool,
196 AsyncProcess.AsyncProcessCallback<R> callback,
197 Configuration confn ) {
198 ap = new MyAsyncProcess<R>(this, callback, c, nbThreads);
199 return ap;
200 }
201
202 @Override
203 public HRegionLocation locateRegion(final TableName tableName,
204 final byte[] row) {
205 return loc1;
206 }
207 }
208
209
210
211
212 static class MyConnectionImpl2 extends MyConnectionImpl {
213 List<HRegionLocation> hrl;
214 final boolean usedRegions[];
215
216 protected MyConnectionImpl2(List<HRegionLocation> hrl) {
217 super(c);
218 this.hrl = hrl;
219 this.usedRegions = new boolean[hrl.size()];
220 }
221
222 @Override
223 public HRegionLocation locateRegion(final TableName tableName,
224 final byte[] row) {
225 int i = 0;
226 for (HRegionLocation hr:hrl){
227 if (Arrays.equals(row, hr.getRegionInfo().getStartKey())){
228 usedRegions[i] = true;
229 return hr;
230 }
231 i++;
232 }
233 return null;
234 }
235 }
236
237 @Test
238 public void testSubmit() throws Exception {
239 HConnection hc = createHConnection();
240 AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
241
242 List<Put> puts = new ArrayList<Put>();
243 puts.add(createPut(1, true));
244
245 ap.submit(puts, false);
246 Assert.assertTrue(puts.isEmpty());
247 }
248
249 @Test
250 public void testSubmitWithCB() throws Exception {
251 HConnection hc = createHConnection();
252 MyCB mcb = new MyCB();
253 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
254
255 List<Put> puts = new ArrayList<Put>();
256 puts.add(createPut(1, true));
257
258 ap.submit(puts, false);
259 Assert.assertTrue(puts.isEmpty());
260
261 while (!(mcb.successCalled.get() == 1) && !ap.hasError()) {
262 Thread.sleep(1);
263 }
264 Assert.assertEquals(mcb.successCalled.get(), 1);
265 }
266
267 @Test
268 public void testSubmitBusyRegion() throws Exception {
269 HConnection hc = createHConnection();
270 AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
271
272 List<Put> puts = new ArrayList<Put>();
273 puts.add(createPut(1, true));
274
275 ap.incTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
276 ap.submit(puts, false);
277 Assert.assertEquals(puts.size(), 1);
278
279 ap.decTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
280 ap.submit(puts, false);
281 Assert.assertTrue(puts.isEmpty());
282 }
283
284
285 @Test
286 public void testSubmitBusyRegionServer() throws Exception {
287 HConnection hc = createHConnection();
288 AsyncProcess<Object> ap = new MyAsyncProcess<Object>(hc, null, conf);
289
290 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer));
291
292 List<Put> puts = new ArrayList<Put>();
293 puts.add(createPut(1, true));
294 puts.add(createPut(3, true));
295 puts.add(createPut(1, true));
296 puts.add(createPut(2, true));
297
298 ap.submit(puts, false);
299 Assert.assertEquals(" puts=" + puts, 1, puts.size());
300
301 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer - 1));
302 ap.submit(puts, false);
303 Assert.assertTrue(puts.isEmpty());
304 }
305
306 @Test
307 public void testFail() throws Exception {
308 HConnection hc = createHConnection();
309 MyCB mcb = new MyCB();
310 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
311
312 List<Put> puts = new ArrayList<Put>();
313 Put p = createPut(1, false);
314 puts.add(p);
315
316 ap.submit(puts, false);
317 Assert.assertTrue(puts.isEmpty());
318
319 while (!ap.hasError()) {
320 Thread.sleep(1);
321 }
322
323 Assert.assertEquals(0, mcb.successCalled.get());
324 Assert.assertEquals(2, mcb.retriableFailure.get());
325 Assert.assertEquals(1, mcb.failureCalled.get());
326
327 Assert.assertEquals(1, ap.getErrors().exceptions.size());
328 Assert.assertTrue("was: " + ap.getErrors().exceptions.get(0),
329 failure.equals(ap.getErrors().exceptions.get(0)));
330 Assert.assertTrue("was: " + ap.getErrors().exceptions.get(0),
331 failure.equals(ap.getErrors().exceptions.get(0)));
332
333 Assert.assertEquals(1, ap.getFailedOperations().size());
334 Assert.assertTrue("was: " + ap.getFailedOperations().get(0),
335 p.equals(ap.getFailedOperations().get(0)));
336 }
337
338 @Test
339 public void testWaitForNextTaskDone() throws IOException {
340 HConnection hc = createHConnection();
341 MyCB mcb = new MyCB();
342 final AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
343 ap.tasksSent.incrementAndGet();
344
345 final AtomicBoolean checkPoint = new AtomicBoolean(false);
346 final AtomicBoolean checkPoint2 = new AtomicBoolean(false);
347
348 Thread t = new Thread(){
349 @Override
350 public void run(){
351 Threads.sleep(1000);
352 Assert.assertFalse(checkPoint.get());
353 ap.tasksDone.incrementAndGet();
354 checkPoint2.set(true);
355 }
356 };
357
358 t.start();
359 ap.waitForNextTaskDone(0);
360 checkPoint.set(true);
361 while (!checkPoint2.get()){
362 Threads.sleep(1);
363 }
364 }
365
366 @Test
367 public void testSubmitTrue() throws IOException {
368 HConnection hc = createHConnection();
369 MyCB mcb = new MyCB();
370 final AsyncProcess<Object> ap = new MyAsyncProcess<Object>(hc, mcb, conf);
371 ap.tasksSent.incrementAndGet();
372 final AtomicInteger ai = new AtomicInteger(1);
373 ap.taskCounterPerRegion.put(hri1.getRegionName(), ai);
374
375 final AtomicBoolean checkPoint = new AtomicBoolean(false);
376 final AtomicBoolean checkPoint2 = new AtomicBoolean(false);
377
378 Thread t = new Thread(){
379 @Override
380 public void run(){
381 Threads.sleep(1000);
382 Assert.assertFalse(checkPoint.get());
383 ai.decrementAndGet();
384 ap.tasksDone.incrementAndGet();
385 checkPoint2.set(true);
386 }
387 };
388
389 List<Put> puts = new ArrayList<Put>();
390 Put p = createPut(1, true);
391 puts.add(p);
392
393 ap.submit(puts, false);
394 Assert.assertFalse(puts.isEmpty());
395
396 t.start();
397
398 ap.submit(puts, true);
399 Assert.assertTrue(puts.isEmpty());
400
401 checkPoint.set(true);
402 while (!checkPoint2.get()){
403 Threads.sleep(1);
404 }
405 }
406
407 @Test
408 public void testFailAndSuccess() throws Exception {
409 HConnection hc = createHConnection();
410 MyCB mcb = new MyCB();
411 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
412
413 List<Put> puts = new ArrayList<Put>();
414 puts.add(createPut(1, false));
415 puts.add(createPut(1, true));
416 puts.add(createPut(1, true));
417
418 ap.submit(puts, false);
419 Assert.assertTrue(puts.isEmpty());
420
421 while (!ap.hasError()) {
422 Thread.sleep(1);
423 }
424 ap.waitUntilDone();
425
426 Assert.assertEquals(mcb.successCalled.get(), 2);
427 Assert.assertEquals(mcb.retriableFailure.get(), 2);
428 Assert.assertEquals(mcb.failureCalled.get(), 1);
429
430 Assert.assertEquals(1, ap.getErrors().actions.size());
431
432
433 puts.add(createPut(1, true));
434 ap.submit(puts, false);
435 Assert.assertTrue(puts.isEmpty());
436
437 while (mcb.successCalled.get() != 3) {
438 Thread.sleep(1);
439 }
440 Assert.assertEquals(mcb.retriableFailure.get(), 2);
441 Assert.assertEquals(mcb.failureCalled.get(), 1);
442
443 ap.clearErrors();
444 Assert.assertTrue(ap.getErrors().actions.isEmpty());
445 }
446
447 @Test
448 public void testFlush() throws Exception {
449 HConnection hc = createHConnection();
450 MyCB mcb = new MyCB();
451 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
452
453 List<Put> puts = new ArrayList<Put>();
454 puts.add(createPut(1, false));
455 puts.add(createPut(1, true));
456 puts.add(createPut(1, true));
457
458 ap.submit(puts, false);
459 ap.waitUntilDone();
460
461 Assert.assertEquals(mcb.successCalled.get(), 2);
462 Assert.assertEquals(mcb.retriableFailure.get(), 2);
463 Assert.assertEquals(mcb.failureCalled.get(), 1);
464
465 Assert.assertEquals(1, ap.getFailedOperations().size());
466 }
467
468 @Test
469 public void testMaxTask() throws Exception {
470 HConnection hc = createHConnection();
471 final AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
472
473 for (int i = 0; i < 1000; i++) {
474 ap.incTaskCounters(Arrays.asList("dummy".getBytes()), sn);
475 }
476
477 final Thread myThread = Thread.currentThread();
478
479 Thread t = new Thread() {
480 public void run() {
481 Threads.sleep(2000);
482 myThread.interrupt();
483 }
484 };
485
486 List<Put> puts = new ArrayList<Put>();
487 puts.add(createPut(1, true));
488
489 t.start();
490
491 try {
492 ap.submit(puts, false);
493 Assert.fail("We should have been interrupted.");
494 } catch (InterruptedIOException expected) {
495 }
496
497 final long sleepTime = 2000;
498
499 Thread t2 = new Thread() {
500 public void run() {
501 Threads.sleep(sleepTime);
502 while (ap.tasksDone.get() > 0) {
503 ap.decTaskCounters(Arrays.asList("dummy".getBytes()), sn);
504 }
505 }
506 };
507 t2.start();
508
509 long start = System.currentTimeMillis();
510 ap.submit(new ArrayList<Row>(), false);
511 long end = System.currentTimeMillis();
512
513
514 Assert.assertTrue(start + 100L + sleepTime > end);
515 }
516
517
518 private class MyCB implements AsyncProcess.AsyncProcessCallback<Object> {
519 private final AtomicInteger successCalled = new AtomicInteger(0);
520 private final AtomicInteger failureCalled = new AtomicInteger(0);
521 private final AtomicInteger retriableFailure = new AtomicInteger(0);
522
523
524 @Override
525 public void success(int originalIndex, byte[] region, Row row, Object o) {
526 successCalled.incrementAndGet();
527 }
528
529 @Override
530 public boolean failure(int originalIndex, byte[] region, Row row, Throwable t) {
531 failureCalled.incrementAndGet();
532 return true;
533 }
534
535 @Override
536 public boolean retriableFailure(int originalIndex, Row row, byte[] region,
537 Throwable exception) {
538
539 return (retriableFailure.incrementAndGet() < 2);
540 }
541 }
542
543
544 private static HConnection createHConnection() throws IOException {
545 HConnection hc = Mockito.mock(HConnection.class);
546
547 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
548 Mockito.eq(DUMMY_BYTES_1), Mockito.anyBoolean())).thenReturn(loc1);
549 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
550 Mockito.eq(DUMMY_BYTES_1))).thenReturn(loc1);
551
552 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
553 Mockito.eq(DUMMY_BYTES_2), Mockito.anyBoolean())).thenReturn(loc2);
554 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
555 Mockito.eq(DUMMY_BYTES_2))).thenReturn(loc2);
556
557 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
558 Mockito.eq(DUMMY_BYTES_3), Mockito.anyBoolean())).thenReturn(loc2);
559 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
560 Mockito.eq(DUMMY_BYTES_3))).thenReturn(loc3);
561
562 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
563 Mockito.eq(FAILS), Mockito.anyBoolean())).thenReturn(loc2);
564 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
565 Mockito.eq(FAILS))).thenReturn(loc2);
566
567 return hc;
568 }
569
570 @Test
571 public void testHTablePutSuccess() throws Exception {
572 HTable ht = Mockito.mock(HTable.class);
573 HConnection hc = createHConnection();
574 ht.ap = new MyAsyncProcess<Object>(hc, null, conf);
575
576 Put put = createPut(1, true);
577
578 Assert.assertEquals(0, ht.getWriteBufferSize());
579 ht.put(put);
580 Assert.assertEquals(0, ht.getWriteBufferSize());
581 }
582
583 private void doHTableFailedPut(boolean bufferOn) throws Exception {
584 HTable ht = new HTable();
585 HConnection hc = createHConnection();
586 MyCB mcb = new MyCB();
587 ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
588 ht.setAutoFlush(true, true);
589 if (bufferOn) {
590 ht.setWriteBufferSize(1024L * 1024L);
591 } else {
592 ht.setWriteBufferSize(0L);
593 }
594
595 Put put = createPut(1, false);
596
597 Assert.assertEquals(0L, ht.currentWriteBufferSize);
598 try {
599 ht.put(put);
600 if (bufferOn) {
601 ht.flushCommits();
602 }
603 Assert.fail();
604 } catch (RetriesExhaustedException expected) {
605 }
606 Assert.assertEquals(0L, ht.currentWriteBufferSize);
607 Assert.assertEquals(0, mcb.successCalled.get());
608 Assert.assertEquals(2, mcb.retriableFailure.get());
609 Assert.assertEquals(1, mcb.failureCalled.get());
610
611
612 ht.close();
613 }
614
615 @Test
616 public void testHTableFailedPutWithBuffer() throws Exception {
617 doHTableFailedPut(true);
618 }
619
620 @Test
621 public void doHTableFailedPutWithoutBuffer() throws Exception {
622 doHTableFailedPut(false);
623 }
624
625 @Test
626 public void testHTableFailedPutAndNewPut() throws Exception {
627 HTable ht = new HTable();
628 HConnection hc = createHConnection();
629 MyCB mcb = new MyCB();
630 ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
631 ht.setAutoFlush(false, true);
632 ht.setWriteBufferSize(0);
633
634 Put p = createPut(1, false);
635 ht.put(p);
636
637 ht.ap.waitUntilDone();
638
639
640
641
642
643
644 p = createPut(1, true);
645 Assert.assertEquals(0, ht.writeAsyncBuffer.size());
646 try {
647 ht.put(p);
648 Assert.fail();
649 } catch (RetriesExhaustedException expected) {
650 }
651 Assert.assertEquals("the put should not been inserted.", 0, ht.writeAsyncBuffer.size());
652 }
653
654
655 @Test
656 public void testWithNoClearOnFail() throws IOException {
657 HTable ht = new HTable();
658 HConnection hc = createHConnection();
659 MyCB mcb = new MyCB();
660 ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
661 ht.setAutoFlush(false, false);
662
663 Put p = createPut(1, false);
664 ht.put(p);
665 Assert.assertEquals(0, ht.writeAsyncBuffer.size());
666 try {
667 ht.flushCommits();
668 } catch (RetriesExhaustedWithDetailsException expected) {
669 }
670 Assert.assertEquals(1, ht.writeAsyncBuffer.size());
671
672 try {
673 ht.close();
674 } catch (RetriesExhaustedWithDetailsException expected) {
675 }
676 Assert.assertEquals(1, ht.writeAsyncBuffer.size());
677 }
678
679 @Test
680 public void testBatch() throws IOException, InterruptedException {
681 HTable ht = new HTable();
682 ht.connection = new MyConnectionImpl();
683
684 List<Put> puts = new ArrayList<Put>();
685 puts.add(createPut(1, true));
686 puts.add(createPut(1, true));
687 puts.add(createPut(1, true));
688 puts.add(createPut(1, true));
689 puts.add(createPut(1, false));
690 puts.add(createPut(1, true));
691 puts.add(createPut(1, false));
692
693 Object[] res = new Object[puts.size()];
694 try {
695 ht.processBatch(puts, res);
696 Assert.fail();
697 } catch (RetriesExhaustedException expected) {
698 }
699
700 Assert.assertEquals(res[0], success);
701 Assert.assertEquals(res[1], success);
702 Assert.assertEquals(res[2], success);
703 Assert.assertEquals(res[3], success);
704 Assert.assertEquals(res[4], failure);
705 Assert.assertEquals(res[5], success);
706 Assert.assertEquals(res[6], failure);
707 }
708
709 @Test
710 public void testErrorsServers() throws IOException {
711 HTable ht = new HTable();
712 Configuration configuration = new Configuration(conf);
713 configuration.setBoolean(HConnectionManager.RETRIES_BY_SERVER_KEY, true);
714 configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
715
716 ht.setWriteBufferSize(configuration.getLong("hbase.client.write.buffer", 2097152));
717
718 MyConnectionImpl mci = new MyConnectionImpl(configuration);
719 ht.connection = mci;
720 ht.ap = new MyAsyncProcess<Object>(mci, null, configuration);
721
722
723 Assert.assertNotNull(ht.ap.createServerErrorTracker());
724 Assert.assertTrue(ht.ap.serverTrackerTimeout > 200);
725 ht.ap.serverTrackerTimeout = 1;
726
727 Put p = createPut(1, false);
728 ht.setAutoFlush(false, false);
729 ht.put(p);
730
731 try {
732 ht.flushCommits();
733 Assert.fail();
734 } catch (RetriesExhaustedWithDetailsException expected) {
735 }
736
737 Assert.assertEquals(ht.ap.tasksSent.get(), 3);
738 }
739
740 @Test
741 public void testGlobalErrors() throws IOException {
742 HTable ht = new HTable();
743 Configuration configuration = new Configuration(conf);
744 configuration.setBoolean(HConnectionManager.RETRIES_BY_SERVER_KEY, true);
745 configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
746 ht.connection = new MyConnectionImpl(configuration);
747 AsyncProcessWithFailure<Object> ap =
748 new AsyncProcessWithFailure<Object>(ht.connection, configuration);
749 ht.ap = ap;
750
751 Assert.assertNotNull(ht.ap.createServerErrorTracker());
752
753 Put p = createPut(1, true);
754 ht.setAutoFlush(false, false);
755 ht.put(p);
756
757 try {
758 ht.flushCommits();
759 Assert.fail();
760 } catch (RetriesExhaustedWithDetailsException expected) {
761 }
762
763 Assert.assertEquals(3, ht.ap.tasksSent.get());
764 }
765
766
767
768
769
770
771 @Test
772 public void testThreadCreation() throws Exception {
773 final int NB_REGS = 100;
774 List<HRegionLocation> hrls = new ArrayList<HRegionLocation>(NB_REGS);
775 List<Get> gets = new ArrayList<Get>(NB_REGS);
776 for (int i = 0; i < NB_REGS; i++) {
777 HRegionInfo hri = new HRegionInfo(
778 DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i);
779 HRegionLocation hrl = new HRegionLocation(hri, i % 2 == 0 ? sn : sn2);
780 hrls.add(hrl);
781
782 Get get = new Get(Bytes.toBytes(i * 10L));
783 gets.add(get);
784 }
785
786 HTable ht = new HTable();
787 MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
788 ht.connection = con;
789
790 ht.batch(gets);
791
792 Assert.assertEquals(con.ap.nbActions.get(), NB_REGS);
793 Assert.assertEquals("1 multi response per server", 2, con.ap.nbMultiResponse.get());
794 Assert.assertEquals("1 thread per server", 2, con.nbThreads.get());
795
796 int nbReg = 0;
797 for (int i =0; i<NB_REGS; i++){
798 if (con.usedRegions[i]) nbReg++;
799 }
800 Assert.assertEquals("nbReg=" + nbReg, nbReg, NB_REGS);
801 }
802
803
804
805
806
807
808
809 private Put createPut(int regCnt, boolean success) {
810 Put p;
811 if (!success) {
812 p = new Put(FAILS);
813 } else switch (regCnt){
814 case 1 :
815 p = new Put(DUMMY_BYTES_1);
816 break;
817 case 2:
818 p = new Put(DUMMY_BYTES_2);
819 break;
820 case 3:
821 p = new Put(DUMMY_BYTES_3);
822 break;
823 default:
824 throw new IllegalArgumentException("unknown " + regCnt);
825 }
826
827 p.add(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
828
829 return p;
830 }
831
832 static class MyThreadPoolExecutor extends ThreadPoolExecutor {
833 public MyThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime,
834 TimeUnit timeunit, BlockingQueue<Runnable> blockingqueue) {
835 super(coreThreads, maxThreads, keepAliveTime, timeunit, blockingqueue);
836 }
837
838 @Override
839 public Future submit(Runnable runnable) {
840 throw new OutOfMemoryError("OutOfMemory error thrown by means");
841 }
842 }
843
844 static class AsyncProcessForThrowableCheck extends AsyncProcess {
845 public AsyncProcessForThrowableCheck(HConnection hc, Configuration conf, ExecutorService pool) {
846 super(hc, DUMMY_TABLE, pool, null, conf, new RpcRetryingCallerFactory(conf),
847 new RpcControllerFactory(conf));
848 }
849 }
850
851 @Test
852 public void testUncheckedException() throws Exception {
853
854 HConnection hc = createHConnection();
855 MyThreadPoolExecutor myPool =
856 new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
857 new LinkedBlockingQueue<Runnable>(200));
858 Configuration myConf = new Configuration(conf);
859 myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
860 AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, myConf, myPool);
861
862 List<Put> puts = new ArrayList<Put>();
863 puts.add(createPut(1, true));
864
865 ap.submit(puts, false, null);
866 Assert.assertTrue(puts.isEmpty());
867 }
868 }