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 @Override
209 public boolean hasCellBlockSupport() {
210 return false;
211 }
212 }
213
214
215
216
217 static class MyConnectionImpl2 extends MyConnectionImpl {
218 List<HRegionLocation> hrl;
219 final boolean usedRegions[];
220
221 protected MyConnectionImpl2(List<HRegionLocation> hrl) {
222 super(c);
223 this.hrl = hrl;
224 this.usedRegions = new boolean[hrl.size()];
225 }
226
227 @Override
228 public HRegionLocation locateRegion(final TableName tableName,
229 final byte[] row) {
230 int i = 0;
231 for (HRegionLocation hr:hrl){
232 if (Arrays.equals(row, hr.getRegionInfo().getStartKey())){
233 usedRegions[i] = true;
234 return hr;
235 }
236 i++;
237 }
238 return null;
239 }
240 }
241
242 @Test
243 public void testSubmit() throws Exception {
244 HConnection hc = createHConnection();
245 AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
246
247 List<Put> puts = new ArrayList<Put>();
248 puts.add(createPut(1, true));
249
250 ap.submit(puts, false);
251 Assert.assertTrue(puts.isEmpty());
252 }
253
254 @Test
255 public void testSubmitWithCB() throws Exception {
256 HConnection hc = createHConnection();
257 MyCB mcb = new MyCB();
258 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
259
260 List<Put> puts = new ArrayList<Put>();
261 puts.add(createPut(1, true));
262
263 ap.submit(puts, false);
264 Assert.assertTrue(puts.isEmpty());
265
266 while (!(mcb.successCalled.get() == 1) && !ap.hasError()) {
267 Thread.sleep(1);
268 }
269 Assert.assertEquals(mcb.successCalled.get(), 1);
270 }
271
272 @Test
273 public void testSubmitBusyRegion() throws Exception {
274 HConnection hc = createHConnection();
275 AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
276
277 List<Put> puts = new ArrayList<Put>();
278 puts.add(createPut(1, true));
279
280 ap.incTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
281 ap.submit(puts, false);
282 Assert.assertEquals(puts.size(), 1);
283
284 ap.decTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
285 ap.submit(puts, false);
286 Assert.assertTrue(puts.isEmpty());
287 }
288
289
290 @Test
291 public void testSubmitBusyRegionServer() throws Exception {
292 HConnection hc = createHConnection();
293 AsyncProcess<Object> ap = new MyAsyncProcess<Object>(hc, null, conf);
294
295 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer));
296
297 List<Put> puts = new ArrayList<Put>();
298 puts.add(createPut(1, true));
299 puts.add(createPut(3, true));
300 puts.add(createPut(1, true));
301 puts.add(createPut(2, true));
302
303 ap.submit(puts, false);
304 Assert.assertEquals(" puts=" + puts, 1, puts.size());
305
306 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer - 1));
307 ap.submit(puts, false);
308 Assert.assertTrue(puts.isEmpty());
309 }
310
311 @Test
312 public void testFail() throws Exception {
313 HConnection hc = createHConnection();
314 MyCB mcb = new MyCB();
315 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
316
317 List<Put> puts = new ArrayList<Put>();
318 Put p = createPut(1, false);
319 puts.add(p);
320
321 ap.submit(puts, false);
322 Assert.assertTrue(puts.isEmpty());
323
324 while (!ap.hasError()) {
325 Thread.sleep(1);
326 }
327
328 Assert.assertEquals(0, mcb.successCalled.get());
329 Assert.assertEquals(2, mcb.retriableFailure.get());
330 Assert.assertEquals(1, mcb.failureCalled.get());
331
332 Assert.assertEquals(1, ap.getErrors().exceptions.size());
333 Assert.assertTrue("was: " + ap.getErrors().exceptions.get(0),
334 failure.equals(ap.getErrors().exceptions.get(0)));
335 Assert.assertTrue("was: " + ap.getErrors().exceptions.get(0),
336 failure.equals(ap.getErrors().exceptions.get(0)));
337
338 Assert.assertEquals(1, ap.getFailedOperations().size());
339 Assert.assertTrue("was: " + ap.getFailedOperations().get(0),
340 p.equals(ap.getFailedOperations().get(0)));
341 }
342
343 @Test
344 public void testWaitForNextTaskDone() throws IOException {
345 HConnection hc = createHConnection();
346 MyCB mcb = new MyCB();
347 final AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
348 ap.tasksSent.incrementAndGet();
349
350 final AtomicBoolean checkPoint = new AtomicBoolean(false);
351 final AtomicBoolean checkPoint2 = new AtomicBoolean(false);
352
353 Thread t = new Thread(){
354 @Override
355 public void run(){
356 Threads.sleep(1000);
357 Assert.assertFalse(checkPoint.get());
358 ap.tasksDone.incrementAndGet();
359 checkPoint2.set(true);
360 }
361 };
362
363 t.start();
364 ap.waitForNextTaskDone(0);
365 checkPoint.set(true);
366 while (!checkPoint2.get()){
367 Threads.sleep(1);
368 }
369 }
370
371 @Test
372 public void testSubmitTrue() throws IOException {
373 HConnection hc = createHConnection();
374 MyCB mcb = new MyCB();
375 final AsyncProcess<Object> ap = new MyAsyncProcess<Object>(hc, mcb, conf);
376 ap.tasksSent.incrementAndGet();
377 final AtomicInteger ai = new AtomicInteger(1);
378 ap.taskCounterPerRegion.put(hri1.getRegionName(), ai);
379
380 final AtomicBoolean checkPoint = new AtomicBoolean(false);
381 final AtomicBoolean checkPoint2 = new AtomicBoolean(false);
382
383 Thread t = new Thread(){
384 @Override
385 public void run(){
386 Threads.sleep(1000);
387 Assert.assertFalse(checkPoint.get());
388 ai.decrementAndGet();
389 ap.tasksDone.incrementAndGet();
390 checkPoint2.set(true);
391 }
392 };
393
394 List<Put> puts = new ArrayList<Put>();
395 Put p = createPut(1, true);
396 puts.add(p);
397
398 ap.submit(puts, false);
399 Assert.assertFalse(puts.isEmpty());
400
401 t.start();
402
403 ap.submit(puts, true);
404 Assert.assertTrue(puts.isEmpty());
405
406 checkPoint.set(true);
407 while (!checkPoint2.get()){
408 Threads.sleep(1);
409 }
410 }
411
412 @Test
413 public void testFailAndSuccess() throws Exception {
414 HConnection hc = createHConnection();
415 MyCB mcb = new MyCB();
416 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
417
418 List<Put> puts = new ArrayList<Put>();
419 puts.add(createPut(1, false));
420 puts.add(createPut(1, true));
421 puts.add(createPut(1, true));
422
423 ap.submit(puts, false);
424 Assert.assertTrue(puts.isEmpty());
425
426 while (!ap.hasError()) {
427 Thread.sleep(1);
428 }
429 ap.waitUntilDone();
430
431 Assert.assertEquals(mcb.successCalled.get(), 2);
432 Assert.assertEquals(mcb.retriableFailure.get(), 2);
433 Assert.assertEquals(mcb.failureCalled.get(), 1);
434
435 Assert.assertEquals(1, ap.getErrors().actions.size());
436
437
438 puts.add(createPut(1, true));
439 ap.submit(puts, false);
440 Assert.assertTrue(puts.isEmpty());
441
442 while (mcb.successCalled.get() != 3) {
443 Thread.sleep(1);
444 }
445 Assert.assertEquals(mcb.retriableFailure.get(), 2);
446 Assert.assertEquals(mcb.failureCalled.get(), 1);
447
448 ap.clearErrors();
449 Assert.assertTrue(ap.getErrors().actions.isEmpty());
450 }
451
452 @Test
453 public void testFlush() throws Exception {
454 HConnection hc = createHConnection();
455 MyCB mcb = new MyCB();
456 AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
457
458 List<Put> puts = new ArrayList<Put>();
459 puts.add(createPut(1, false));
460 puts.add(createPut(1, true));
461 puts.add(createPut(1, true));
462
463 ap.submit(puts, false);
464 ap.waitUntilDone();
465
466 Assert.assertEquals(mcb.successCalled.get(), 2);
467 Assert.assertEquals(mcb.retriableFailure.get(), 2);
468 Assert.assertEquals(mcb.failureCalled.get(), 1);
469
470 Assert.assertEquals(1, ap.getFailedOperations().size());
471 }
472
473 @Test
474 public void testMaxTask() throws Exception {
475 HConnection hc = createHConnection();
476 final AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
477
478 for (int i = 0; i < 1000; i++) {
479 ap.incTaskCounters(Arrays.asList("dummy".getBytes()), sn);
480 }
481
482 final Thread myThread = Thread.currentThread();
483
484 Thread t = new Thread() {
485 public void run() {
486 Threads.sleep(2000);
487 myThread.interrupt();
488 }
489 };
490
491 List<Put> puts = new ArrayList<Put>();
492 puts.add(createPut(1, true));
493
494 t.start();
495
496 try {
497 ap.submit(puts, false);
498 Assert.fail("We should have been interrupted.");
499 } catch (InterruptedIOException expected) {
500 }
501
502 final long sleepTime = 2000;
503
504 Thread t2 = new Thread() {
505 public void run() {
506 Threads.sleep(sleepTime);
507 while (ap.tasksDone.get() > 0) {
508 ap.decTaskCounters(Arrays.asList("dummy".getBytes()), sn);
509 }
510 }
511 };
512 t2.start();
513
514 long start = System.currentTimeMillis();
515 ap.submit(new ArrayList<Row>(), false);
516 long end = System.currentTimeMillis();
517
518
519 Assert.assertTrue(start + 100L + sleepTime > end);
520 }
521
522
523 private class MyCB implements AsyncProcess.AsyncProcessCallback<Object> {
524 private final AtomicInteger successCalled = new AtomicInteger(0);
525 private final AtomicInteger failureCalled = new AtomicInteger(0);
526 private final AtomicInteger retriableFailure = new AtomicInteger(0);
527
528
529 @Override
530 public void success(int originalIndex, byte[] region, Row row, Object o) {
531 successCalled.incrementAndGet();
532 }
533
534 @Override
535 public boolean failure(int originalIndex, byte[] region, Row row, Throwable t) {
536 failureCalled.incrementAndGet();
537 return true;
538 }
539
540 @Override
541 public boolean retriableFailure(int originalIndex, Row row, byte[] region,
542 Throwable exception) {
543
544 return (retriableFailure.incrementAndGet() < 2);
545 }
546 }
547
548
549 private static HConnection createHConnection() throws IOException {
550 HConnection hc = Mockito.mock(HConnection.class);
551
552 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
553 Mockito.eq(DUMMY_BYTES_1), Mockito.anyBoolean())).thenReturn(loc1);
554 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
555 Mockito.eq(DUMMY_BYTES_1))).thenReturn(loc1);
556
557 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
558 Mockito.eq(DUMMY_BYTES_2), Mockito.anyBoolean())).thenReturn(loc2);
559 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
560 Mockito.eq(DUMMY_BYTES_2))).thenReturn(loc2);
561
562 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
563 Mockito.eq(DUMMY_BYTES_3), Mockito.anyBoolean())).thenReturn(loc2);
564 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
565 Mockito.eq(DUMMY_BYTES_3))).thenReturn(loc3);
566
567 Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
568 Mockito.eq(FAILS), Mockito.anyBoolean())).thenReturn(loc2);
569 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
570 Mockito.eq(FAILS))).thenReturn(loc2);
571
572 return hc;
573 }
574
575 @Test
576 public void testHTablePutSuccess() throws Exception {
577 HTable ht = Mockito.mock(HTable.class);
578 HConnection hc = createHConnection();
579 ht.ap = new MyAsyncProcess<Object>(hc, null, conf);
580
581 Put put = createPut(1, true);
582
583 Assert.assertEquals(0, ht.getWriteBufferSize());
584 ht.put(put);
585 Assert.assertEquals(0, ht.getWriteBufferSize());
586 }
587
588 private void doHTableFailedPut(boolean bufferOn) throws Exception {
589 HTable ht = new HTable();
590 HConnection hc = createHConnection();
591 MyCB mcb = new MyCB();
592 ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
593 ht.setAutoFlush(true, true);
594 if (bufferOn) {
595 ht.setWriteBufferSize(1024L * 1024L);
596 } else {
597 ht.setWriteBufferSize(0L);
598 }
599
600 Put put = createPut(1, false);
601
602 Assert.assertEquals(0L, ht.currentWriteBufferSize);
603 try {
604 ht.put(put);
605 if (bufferOn) {
606 ht.flushCommits();
607 }
608 Assert.fail();
609 } catch (RetriesExhaustedException expected) {
610 }
611 Assert.assertEquals(0L, ht.currentWriteBufferSize);
612 Assert.assertEquals(0, mcb.successCalled.get());
613 Assert.assertEquals(2, mcb.retriableFailure.get());
614 Assert.assertEquals(1, mcb.failureCalled.get());
615
616
617 ht.close();
618 }
619
620 @Test
621 public void testHTableFailedPutWithBuffer() throws Exception {
622 doHTableFailedPut(true);
623 }
624
625 @Test
626 public void doHTableFailedPutWithoutBuffer() throws Exception {
627 doHTableFailedPut(false);
628 }
629
630 @Test
631 public void testHTableFailedPutAndNewPut() throws Exception {
632 HTable ht = new HTable();
633 HConnection hc = createHConnection();
634 MyCB mcb = new MyCB();
635 ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
636 ht.setAutoFlush(false, true);
637 ht.setWriteBufferSize(0);
638
639 Put p = createPut(1, false);
640 ht.put(p);
641
642 ht.ap.waitUntilDone();
643
644
645
646
647
648
649 p = createPut(1, true);
650 Assert.assertEquals(0, ht.writeAsyncBuffer.size());
651 try {
652 ht.put(p);
653 Assert.fail();
654 } catch (RetriesExhaustedException expected) {
655 }
656 Assert.assertEquals("the put should not been inserted.", 0, ht.writeAsyncBuffer.size());
657 }
658
659
660 @Test
661 public void testWithNoClearOnFail() throws IOException {
662 HTable ht = new HTable();
663 HConnection hc = createHConnection();
664 MyCB mcb = new MyCB();
665 ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
666 ht.setAutoFlush(false, false);
667
668 Put p = createPut(1, false);
669 ht.put(p);
670 Assert.assertEquals(0, ht.writeAsyncBuffer.size());
671 try {
672 ht.flushCommits();
673 } catch (RetriesExhaustedWithDetailsException expected) {
674 }
675 Assert.assertEquals(1, ht.writeAsyncBuffer.size());
676
677 try {
678 ht.close();
679 } catch (RetriesExhaustedWithDetailsException expected) {
680 }
681 Assert.assertEquals(1, ht.writeAsyncBuffer.size());
682 }
683
684 @Test
685 public void testBatch() throws IOException, InterruptedException {
686 HTable ht = new HTable();
687 ht.connection = new MyConnectionImpl();
688
689 List<Put> puts = new ArrayList<Put>();
690 puts.add(createPut(1, true));
691 puts.add(createPut(1, true));
692 puts.add(createPut(1, true));
693 puts.add(createPut(1, true));
694 puts.add(createPut(1, false));
695 puts.add(createPut(1, true));
696 puts.add(createPut(1, false));
697
698 Object[] res = new Object[puts.size()];
699 try {
700 ht.processBatch(puts, res);
701 Assert.fail();
702 } catch (RetriesExhaustedException expected) {
703 }
704
705 Assert.assertEquals(res[0], success);
706 Assert.assertEquals(res[1], success);
707 Assert.assertEquals(res[2], success);
708 Assert.assertEquals(res[3], success);
709 Assert.assertEquals(res[4], failure);
710 Assert.assertEquals(res[5], success);
711 Assert.assertEquals(res[6], failure);
712 }
713
714 @Test
715 public void testErrorsServers() throws IOException {
716 HTable ht = new HTable();
717 Configuration configuration = new Configuration(conf);
718 configuration.setBoolean(HConnectionManager.RETRIES_BY_SERVER_KEY, true);
719 configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
720
721 ht.setWriteBufferSize(configuration.getLong("hbase.client.write.buffer", 2097152));
722
723 MyConnectionImpl mci = new MyConnectionImpl(configuration);
724 ht.connection = mci;
725 ht.ap = new MyAsyncProcess<Object>(mci, null, configuration);
726
727
728 Assert.assertNotNull(ht.ap.createServerErrorTracker());
729 Assert.assertTrue(ht.ap.serverTrackerTimeout > 200);
730 ht.ap.serverTrackerTimeout = 1;
731
732 Put p = createPut(1, false);
733 ht.setAutoFlush(false, false);
734 ht.put(p);
735
736 try {
737 ht.flushCommits();
738 Assert.fail();
739 } catch (RetriesExhaustedWithDetailsException expected) {
740 }
741
742 Assert.assertEquals(ht.ap.tasksSent.get(), 3);
743 }
744
745 @Test
746 public void testGlobalErrors() throws IOException {
747 HTable ht = new HTable();
748 Configuration configuration = new Configuration(conf);
749 configuration.setBoolean(HConnectionManager.RETRIES_BY_SERVER_KEY, true);
750 configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
751 ht.connection = new MyConnectionImpl(configuration);
752 AsyncProcessWithFailure<Object> ap =
753 new AsyncProcessWithFailure<Object>(ht.connection, configuration);
754 ht.ap = ap;
755
756 Assert.assertNotNull(ht.ap.createServerErrorTracker());
757
758 Put p = createPut(1, true);
759 ht.setAutoFlush(false, false);
760 ht.put(p);
761
762 try {
763 ht.flushCommits();
764 Assert.fail();
765 } catch (RetriesExhaustedWithDetailsException expected) {
766 }
767
768 Assert.assertEquals(3, ht.ap.tasksSent.get());
769 }
770
771
772
773
774
775
776 @Test
777 public void testThreadCreation() throws Exception {
778 final int NB_REGS = 100;
779 List<HRegionLocation> hrls = new ArrayList<HRegionLocation>(NB_REGS);
780 List<Get> gets = new ArrayList<Get>(NB_REGS);
781 for (int i = 0; i < NB_REGS; i++) {
782 HRegionInfo hri = new HRegionInfo(
783 DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i);
784 HRegionLocation hrl = new HRegionLocation(hri, i % 2 == 0 ? sn : sn2);
785 hrls.add(hrl);
786
787 Get get = new Get(Bytes.toBytes(i * 10L));
788 gets.add(get);
789 }
790
791 HTable ht = new HTable();
792 MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
793 ht.connection = con;
794
795 ht.batch(gets);
796
797 Assert.assertEquals(con.ap.nbActions.get(), NB_REGS);
798 Assert.assertEquals("1 multi response per server", 2, con.ap.nbMultiResponse.get());
799 Assert.assertEquals("1 thread per server", 2, con.nbThreads.get());
800
801 int nbReg = 0;
802 for (int i =0; i<NB_REGS; i++){
803 if (con.usedRegions[i]) nbReg++;
804 }
805 Assert.assertEquals("nbReg=" + nbReg, nbReg, NB_REGS);
806 }
807
808
809
810
811
812
813
814 private Put createPut(int regCnt, boolean success) {
815 Put p;
816 if (!success) {
817 p = new Put(FAILS);
818 } else switch (regCnt){
819 case 1 :
820 p = new Put(DUMMY_BYTES_1);
821 break;
822 case 2:
823 p = new Put(DUMMY_BYTES_2);
824 break;
825 case 3:
826 p = new Put(DUMMY_BYTES_3);
827 break;
828 default:
829 throw new IllegalArgumentException("unknown " + regCnt);
830 }
831
832 p.add(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
833
834 return p;
835 }
836
837 static class MyThreadPoolExecutor extends ThreadPoolExecutor {
838 public MyThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime,
839 TimeUnit timeunit, BlockingQueue<Runnable> blockingqueue) {
840 super(coreThreads, maxThreads, keepAliveTime, timeunit, blockingqueue);
841 }
842
843 @Override
844 public Future submit(Runnable runnable) {
845 throw new OutOfMemoryError("OutOfMemory error thrown by means");
846 }
847 }
848
849 static class AsyncProcessForThrowableCheck extends AsyncProcess {
850 public AsyncProcessForThrowableCheck(HConnection hc, Configuration conf, ExecutorService pool) {
851 super(hc, DUMMY_TABLE, pool, null, conf, new RpcRetryingCallerFactory(conf),
852 new RpcControllerFactory(conf));
853 }
854 }
855
856 @Test
857 public void testUncheckedException() throws Exception {
858
859 HConnection hc = createHConnection();
860 MyThreadPoolExecutor myPool =
861 new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
862 new LinkedBlockingQueue<Runnable>(200));
863 Configuration myConf = new Configuration(conf);
864 myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
865 AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, myConf, myPool);
866
867 List<Put> puts = new ArrayList<Put>();
868 puts.add(createPut(1, true));
869
870 ap.submit(puts, false, null);
871 Assert.assertTrue(puts.isEmpty());
872 }
873 }