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