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