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 java.io.IOException;
24 import java.io.InterruptedIOException;
25 import java.util.ArrayList;
26 import java.util.Arrays;
27 import java.util.HashMap;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.Set;
31 import java.util.TreeSet;
32 import java.util.concurrent.BlockingQueue;
33 import java.util.concurrent.BrokenBarrierException;
34 import java.util.concurrent.CyclicBarrier;
35 import java.util.concurrent.ExecutorService;
36 import java.util.concurrent.Future;
37 import java.util.concurrent.LinkedBlockingQueue;
38 import java.util.concurrent.RejectedExecutionException;
39 import java.util.concurrent.SynchronousQueue;
40 import java.util.concurrent.ThreadFactory;
41 import java.util.concurrent.ThreadPoolExecutor;
42 import java.util.concurrent.TimeUnit;
43 import java.util.concurrent.atomic.AtomicBoolean;
44 import java.util.concurrent.atomic.AtomicInteger;
45 import java.util.concurrent.atomic.AtomicLong;
46
47 import org.apache.commons.logging.Log;
48 import org.apache.commons.logging.LogFactory;
49 import org.apache.hadoop.conf.Configuration;
50 import org.apache.hadoop.hbase.Cell;
51 import org.apache.hadoop.hbase.RegionLocations;
52 import org.apache.hadoop.hbase.TableName;
53 import org.apache.hadoop.hbase.HConstants;
54 import org.apache.hadoop.hbase.HRegionInfo;
55 import org.apache.hadoop.hbase.HRegionLocation;
56 import org.apache.hadoop.hbase.testclassification.MediumTests;
57 import org.apache.hadoop.hbase.ServerName;
58 import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
59 import org.apache.hadoop.hbase.client.coprocessor.Batch;
60 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
61 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
62 import org.apache.hadoop.hbase.util.Bytes;
63 import org.apache.hadoop.hbase.util.Threads;
64 import org.junit.Assert;
65 import static org.junit.Assert.assertTrue;
66 import org.junit.BeforeClass;
67 import org.junit.Rule;
68 import org.junit.Test;
69 import org.junit.experimental.categories.Category;
70 import org.junit.rules.Timeout;
71 import org.mockito.Mockito;
72
73 import java.io.IOException;
74 import java.io.InterruptedIOException;
75 import java.util.ArrayList;
76 import java.util.Arrays;
77 import java.util.HashMap;
78 import java.util.List;
79 import java.util.Map;
80 import java.util.Set;
81 import java.util.TreeSet;
82 import java.util.concurrent.ExecutorService;
83 import java.util.concurrent.RejectedExecutionException;
84 import java.util.concurrent.SynchronousQueue;
85 import java.util.concurrent.ThreadFactory;
86 import java.util.concurrent.ThreadPoolExecutor;
87 import java.util.concurrent.TimeUnit;
88 import java.util.concurrent.atomic.AtomicBoolean;
89 import java.util.concurrent.atomic.AtomicInteger;
90 import java.util.concurrent.atomic.AtomicLong;
91
92 @Category(MediumTests.class)
93 public class TestAsyncProcess {
94 private static final TableName DUMMY_TABLE =
95 TableName.valueOf("DUMMY_TABLE");
96 private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();
97 private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes();
98 private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes();
99 private static final byte[] FAILS = "FAILS".getBytes();
100 private static final Configuration conf = new Configuration();
101
102 private static ServerName sn = ServerName.valueOf("s1:1,1");
103 private static ServerName sn2 = ServerName.valueOf("s2:2,2");
104 private static ServerName sn3 = ServerName.valueOf("s3:3,3");
105 private static HRegionInfo hri1 =
106 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1);
107 private static HRegionInfo hri2 =
108 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_2, HConstants.EMPTY_END_ROW, false, 2);
109 private static HRegionInfo hri3 =
110 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_3, HConstants.EMPTY_END_ROW, false, 3);
111 private static HRegionLocation loc1 = new HRegionLocation(hri1, sn);
112 private static HRegionLocation loc2 = new HRegionLocation(hri2, sn);
113 private static HRegionLocation loc3 = new HRegionLocation(hri3, sn2);
114
115
116 private static HRegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1),
117 hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2);
118 private static HRegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1);
119 private static RegionLocations hrls1 = new RegionLocations(new HRegionLocation(hri1, sn),
120 new HRegionLocation(hri1r1, sn2), new HRegionLocation(hri1r2, sn3));
121 private static RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2),
122 new HRegionLocation(hri2r1, sn3));
123 private static RegionLocations hrls3 = new RegionLocations(new HRegionLocation(hri3, sn3), null);
124
125 private static final String success = "success";
126 private static Exception failure = new Exception("failure");
127
128 private static int NB_RETRIES = 3;
129
130 @BeforeClass
131 public static void beforeClass(){
132 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, NB_RETRIES);
133 }
134
135 static class CountingThreadFactory implements ThreadFactory {
136 final AtomicInteger nbThreads;
137 ThreadFactory realFactory = Threads.newDaemonThreadFactory("test-TestAsyncProcess");
138 @Override
139 public Thread newThread(Runnable r) {
140 nbThreads.incrementAndGet();
141 return realFactory.newThread(r);
142 }
143
144 CountingThreadFactory(AtomicInteger nbThreads){
145 this.nbThreads = nbThreads;
146 }
147 }
148
149 static class MyAsyncProcess extends AsyncProcess {
150 final AtomicInteger nbMultiResponse = new AtomicInteger();
151 final AtomicInteger nbActions = new AtomicInteger();
152 public List<AsyncRequestFuture> allReqs = new ArrayList<AsyncRequestFuture>();
153 public AtomicInteger callsCt = new AtomicInteger();
154
155 @Override
156 protected <Res> AsyncRequestFutureImpl<Res> createAsyncRequestFuture(TableName tableName,
157 List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
158 Batch.Callback<Res> callback, Object[] results, boolean needResults) {
159
160 AsyncRequestFutureImpl<Res> r = super.createAsyncRequestFuture(
161 DUMMY_TABLE, actions, nonceGroup, pool, callback, results, needResults);
162 allReqs.add(r);
163 callsCt.incrementAndGet();
164 return r;
165 }
166
167 public MyAsyncProcess(ClusterConnection hc, Configuration conf) {
168 this(hc, conf, new AtomicInteger());
169 }
170
171 public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) {
172 super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
173 new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
174 new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf));
175 }
176
177 public MyAsyncProcess(
178 ClusterConnection hc, Configuration conf, boolean useGlobalErrors) {
179 super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
180 new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())),
181 new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf));
182 }
183
184 public MyAsyncProcess(ClusterConnection hc, Configuration conf, boolean useGlobalErrors,
185 @SuppressWarnings("unused") boolean dummy) {
186 super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
187 new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())) {
188 @Override
189 public void execute(Runnable command) {
190 throw new RejectedExecutionException("test under failure");
191 }
192 },
193 new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf));
194 }
195
196 @Override
197 public <Res> AsyncRequestFuture submit(TableName tableName, List<? extends Row> rows,
198 boolean atLeastOne, Callback<Res> callback, boolean needResults)
199 throws InterruptedIOException {
200
201 return super.submit(DUMMY_TABLE, rows, atLeastOne, callback, true);
202 }
203
204 @Override
205 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
206 callsCt.incrementAndGet();
207 final MultiResponse mr = createMultiResponse(
208 callable.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() {
209 @Override
210 public void addResponse(MultiResponse mr, byte[] regionName, Action<Row> a) {
211 if (Arrays.equals(FAILS, a.getAction().getRow())) {
212 mr.add(regionName, a.getOriginalIndex(), failure);
213 } else {
214 mr.add(regionName, a.getOriginalIndex(), success);
215 }
216 }
217 });
218
219 return new RpcRetryingCaller<MultiResponse>(100, 10, 9) {
220 @Override
221 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable,
222 int callTimeout)
223 throws IOException, RuntimeException {
224 try {
225
226
227 Thread.sleep(1000);
228 } catch (InterruptedException e) {
229
230 }
231 return mr;
232 }
233 };
234 }
235 }
236
237 static class CallerWithFailure extends RpcRetryingCaller<MultiResponse>{
238
239 public CallerWithFailure() {
240 super(100, 100, 9);
241 }
242
243 @Override
244 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable, int callTimeout)
245 throws IOException, RuntimeException {
246 throw new IOException("test");
247 }
248 }
249
250 static class AsyncProcessWithFailure extends MyAsyncProcess {
251
252 public AsyncProcessWithFailure(ClusterConnection hc, Configuration conf) {
253 super(hc, conf, true);
254 serverTrackerTimeout = 1;
255 }
256
257 @Override
258 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
259 callsCt.incrementAndGet();
260 return new CallerWithFailure();
261 }
262 }
263
264 class MyAsyncProcessWithReplicas extends MyAsyncProcess {
265 private Set<byte[]> failures = new TreeSet<byte[]>(new Bytes.ByteArrayComparator());
266 private long primarySleepMs = 0, replicaSleepMs = 0;
267 private Map<ServerName, Long> customPrimarySleepMs = new HashMap<ServerName, Long>();
268 private final AtomicLong replicaCalls = new AtomicLong(0);
269
270 public void addFailures(HRegionInfo... hris) {
271 for (HRegionInfo hri : hris) {
272 failures.add(hri.getRegionName());
273 }
274 }
275
276 public long getReplicaCallCount() {
277 return replicaCalls.get();
278 }
279
280 public void setPrimaryCallDelay(ServerName server, long primaryMs) {
281 customPrimarySleepMs.put(server, primaryMs);
282 }
283
284 public MyAsyncProcessWithReplicas(ClusterConnection hc, Configuration conf) {
285 super(hc, conf);
286 }
287
288 public void setCallDelays(long primaryMs, long replicaMs) {
289 this.primarySleepMs = primaryMs;
290 this.replicaSleepMs = replicaMs;
291 }
292
293 @Override
294 protected RpcRetryingCaller<MultiResponse> createCaller(
295 MultiServerCallable<Row> callable) {
296 final MultiResponse mr = createMultiResponse(
297 callable.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() {
298 @Override
299 public void addResponse(MultiResponse mr, byte[] regionName, Action<Row> a) {
300 if (failures.contains(regionName)) {
301 mr.add(regionName, a.getOriginalIndex(), failure);
302 } else {
303 boolean isStale = !RegionReplicaUtil.isDefaultReplica(a.getReplicaId());
304 mr.add(regionName, a.getOriginalIndex(),
305 Result.create(new Cell[0], null, isStale));
306 }
307 }
308 });
309
310 final boolean isDefault = RegionReplicaUtil.isDefaultReplica(
311 callable.getMulti().actions.values().iterator().next().iterator().next().getReplicaId());
312 final ServerName server = ((MultiServerCallable<?>)callable).getServerName();
313 String debugMsg = "Call to " + server + ", primary=" + isDefault + " with "
314 + callable.getMulti().actions.size() + " entries: ";
315 for (byte[] region : callable.getMulti().actions.keySet()) {
316 debugMsg += "[" + Bytes.toStringBinary(region) + "], ";
317 }
318 LOG.debug(debugMsg);
319 if (!isDefault) {
320 replicaCalls.incrementAndGet();
321 }
322
323 return new RpcRetryingCaller<MultiResponse>(100, 10, 9) {
324 @Override
325 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable, int callTimeout)
326 throws IOException, RuntimeException {
327 long sleep = -1;
328 if (isDefault) {
329 Long customSleep = customPrimarySleepMs.get(server);
330 sleep = (customSleep == null ? primarySleepMs : customSleep.longValue());
331 } else {
332 sleep = replicaSleepMs;
333 }
334 if (sleep != 0) {
335 try {
336 Thread.sleep(sleep);
337 } catch (InterruptedException e) {
338 }
339 }
340 return mr;
341 }
342 };
343 }
344 }
345
346 static MultiResponse createMultiResponse(final MultiAction<Row> multi,
347 AtomicInteger nbMultiResponse, AtomicInteger nbActions, ResponseGenerator gen) {
348 final MultiResponse mr = new MultiResponse();
349 nbMultiResponse.incrementAndGet();
350 for (Map.Entry<byte[], List<Action<Row>>> entry : multi.actions.entrySet()) {
351 byte[] regionName = entry.getKey();
352 for (Action<Row> a : entry.getValue()) {
353 nbActions.incrementAndGet();
354 gen.addResponse(mr, regionName, a);
355 }
356 }
357 return mr;
358 }
359
360 private static interface ResponseGenerator {
361 void addResponse(final MultiResponse mr, byte[] regionName, Action<Row> a);
362 }
363
364
365
366
367 static class MyConnectionImpl extends ConnectionManager.HConnectionImplementation {
368 final AtomicInteger nbThreads = new AtomicInteger(0);
369
370
371 protected MyConnectionImpl(Configuration conf) {
372 super(conf);
373 }
374
375 @Override
376 public RegionLocations locateRegion(TableName tableName,
377 byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException {
378 return new RegionLocations(loc1);
379 }
380 }
381
382
383
384
385 static class MyConnectionImpl2 extends MyConnectionImpl {
386 List<HRegionLocation> hrl;
387 final boolean usedRegions[];
388
389 protected MyConnectionImpl2(List<HRegionLocation> hrl) {
390 super(conf);
391 this.hrl = hrl;
392 this.usedRegions = new boolean[hrl.size()];
393 }
394
395 @Override
396 public RegionLocations locateRegion(TableName tableName,
397 byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException {
398 int i = 0;
399 for (HRegionLocation hr : hrl){
400 if (Arrays.equals(row, hr.getRegionInfo().getStartKey())) {
401 usedRegions[i] = true;
402 return new RegionLocations(hr);
403 }
404 i++;
405 }
406 return null;
407 }
408
409 }
410
411 @Rule
412 public Timeout timeout = new Timeout(10000);
413
414 @Test
415 public void testSubmit() throws Exception {
416 ClusterConnection hc = createHConnection();
417 AsyncProcess ap = new MyAsyncProcess(hc, conf);
418
419 List<Put> puts = new ArrayList<Put>();
420 puts.add(createPut(1, true));
421
422 ap.submit(DUMMY_TABLE, puts, false, null, false);
423 Assert.assertTrue(puts.isEmpty());
424 }
425
426 @Test
427 public void testSubmitWithCB() throws Exception {
428 ClusterConnection hc = createHConnection();
429 final AtomicInteger updateCalled = new AtomicInteger(0);
430 Batch.Callback<Object> cb = new Batch.Callback<Object>() {
431 @Override
432 public void update(byte[] region, byte[] row, Object result) {
433 updateCalled.incrementAndGet();
434 }
435 };
436 AsyncProcess ap = new MyAsyncProcess(hc, conf);
437
438 List<Put> puts = new ArrayList<Put>();
439 puts.add(createPut(1, true));
440
441 final AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, cb, false);
442 Assert.assertTrue(puts.isEmpty());
443 ars.waitUntilDone();
444 Assert.assertEquals(updateCalled.get(), 1);
445 }
446
447 @Test
448 public void testSubmitBusyRegion() throws Exception {
449 ClusterConnection hc = createHConnection();
450 AsyncProcess ap = new MyAsyncProcess(hc, conf);
451
452 List<Put> puts = new ArrayList<Put>();
453 puts.add(createPut(1, true));
454
455 ap.incTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
456 ap.submit(DUMMY_TABLE, puts, false, null, false);
457 Assert.assertEquals(puts.size(), 1);
458
459 ap.decTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
460 ap.submit(DUMMY_TABLE, puts, false, null, false);
461 Assert.assertEquals(0, puts.size());
462 }
463
464
465 @Test
466 public void testSubmitBusyRegionServer() throws Exception {
467 ClusterConnection hc = createHConnection();
468 AsyncProcess ap = new MyAsyncProcess(hc, conf);
469
470 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer));
471
472 List<Put> puts = new ArrayList<Put>();
473 puts.add(createPut(1, true));
474 puts.add(createPut(3, true));
475 puts.add(createPut(1, true));
476 puts.add(createPut(2, true));
477
478 ap.submit(DUMMY_TABLE, puts, false, null, false);
479 Assert.assertEquals(" puts=" + puts, 1, puts.size());
480
481 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer - 1));
482 ap.submit(DUMMY_TABLE, puts, false, null, false);
483 Assert.assertTrue(puts.isEmpty());
484 }
485
486 @Test
487 public void testFail() throws Exception {
488 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
489
490 List<Put> puts = new ArrayList<Put>();
491 Put p = createPut(1, false);
492 puts.add(p);
493
494 AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
495 Assert.assertEquals(0, puts.size());
496 ars.waitUntilDone();
497 verifyResult(ars, false);
498 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
499
500 Assert.assertEquals(1, ars.getErrors().exceptions.size());
501 Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0),
502 failure.equals(ars.getErrors().exceptions.get(0)));
503 Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0),
504 failure.equals(ars.getErrors().exceptions.get(0)));
505
506 Assert.assertEquals(1, ars.getFailedOperations().size());
507 Assert.assertTrue("was: " + ars.getFailedOperations().get(0),
508 p.equals(ars.getFailedOperations().get(0)));
509 }
510
511
512 @Test
513 public void testSubmitTrue() throws IOException {
514 final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
515 ap.tasksInProgress.incrementAndGet();
516 final AtomicInteger ai = new AtomicInteger(1);
517 ap.taskCounterPerRegion.put(hri1.getRegionName(), ai);
518
519 final AtomicBoolean checkPoint = new AtomicBoolean(false);
520 final AtomicBoolean checkPoint2 = new AtomicBoolean(false);
521
522 Thread t = new Thread(){
523 @Override
524 public void run(){
525 Threads.sleep(1000);
526 Assert.assertFalse(checkPoint.get());
527 ai.decrementAndGet();
528 ap.tasksInProgress.decrementAndGet();
529 checkPoint2.set(true);
530 }
531 };
532
533 List<Put> puts = new ArrayList<Put>();
534 Put p = createPut(1, true);
535 puts.add(p);
536
537 ap.submit(DUMMY_TABLE, puts, false, null, false);
538 Assert.assertFalse(puts.isEmpty());
539
540 t.start();
541
542 ap.submit(DUMMY_TABLE, puts, true, null, false);
543 Assert.assertTrue(puts.isEmpty());
544
545 checkPoint.set(true);
546 while (!checkPoint2.get()){
547 Threads.sleep(1);
548 }
549 }
550
551 @Test
552 public void testFailAndSuccess() throws Exception {
553 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
554
555 List<Put> puts = new ArrayList<Put>();
556 puts.add(createPut(1, false));
557 puts.add(createPut(1, true));
558 puts.add(createPut(1, true));
559
560 AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
561 Assert.assertTrue(puts.isEmpty());
562 ars.waitUntilDone();
563 verifyResult(ars, false, true, true);
564 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
565 ap.callsCt.set(0);
566 Assert.assertEquals(1, ars.getErrors().actions.size());
567
568 puts.add(createPut(1, true));
569
570 ap.waitUntilDone();
571 ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
572 Assert.assertEquals(0, puts.size());
573 ars.waitUntilDone();
574 Assert.assertEquals(2, ap.callsCt.get());
575 verifyResult(ars, true);
576 }
577
578 @Test
579 public void testFlush() throws Exception {
580 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
581
582 List<Put> puts = new ArrayList<Put>();
583 puts.add(createPut(1, false));
584 puts.add(createPut(1, true));
585 puts.add(createPut(1, true));
586
587 AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
588 ars.waitUntilDone();
589 verifyResult(ars, false, true, true);
590 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
591
592 Assert.assertEquals(1, ars.getFailedOperations().size());
593 }
594
595 @Test
596 public void testMaxTask() throws Exception {
597 final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
598
599 for (int i = 0; i < 1000; i++) {
600 ap.incTaskCounters(Arrays.asList("dummy".getBytes()), sn);
601 }
602
603 final Thread myThread = Thread.currentThread();
604
605 Thread t = new Thread() {
606 @Override
607 public void run() {
608 Threads.sleep(2000);
609 myThread.interrupt();
610 }
611 };
612
613 List<Put> puts = new ArrayList<Put>();
614 puts.add(createPut(1, true));
615
616 t.start();
617
618 try {
619 ap.submit(DUMMY_TABLE, puts, false, null, false);
620 Assert.fail("We should have been interrupted.");
621 } catch (InterruptedIOException expected) {
622 }
623
624 final long sleepTime = 2000;
625
626 Thread t2 = new Thread() {
627 @Override
628 public void run() {
629 Threads.sleep(sleepTime);
630 while (ap.tasksInProgress.get() > 0) {
631 ap.decTaskCounters(Arrays.asList("dummy".getBytes()), sn);
632 }
633 }
634 };
635 t2.start();
636
637 long start = System.currentTimeMillis();
638 ap.submit(DUMMY_TABLE, new ArrayList<Row>(), false, null, false);
639 long end = System.currentTimeMillis();
640
641
642 Assert.assertTrue(start + 100L + sleepTime > end);
643 }
644
645 private static ClusterConnection createHConnection() throws IOException {
646 ClusterConnection hc = createHConnectionCommon();
647 setMockLocation(hc, DUMMY_BYTES_1, new RegionLocations(loc1));
648 setMockLocation(hc, DUMMY_BYTES_2, new RegionLocations(loc2));
649 setMockLocation(hc, DUMMY_BYTES_3, new RegionLocations(loc3));
650 setMockLocation(hc, FAILS, new RegionLocations(loc2));
651 return hc;
652 }
653
654 private static ClusterConnection createHConnectionWithReplicas() throws IOException {
655 ClusterConnection hc = createHConnectionCommon();
656 setMockLocation(hc, DUMMY_BYTES_1, hrls1);
657 setMockLocation(hc, DUMMY_BYTES_2, hrls2);
658 setMockLocation(hc, DUMMY_BYTES_3, hrls3);
659 return hc;
660 }
661
662 private static void setMockLocation(ClusterConnection hc, byte[] row,
663 RegionLocations result) throws IOException {
664 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row),
665 Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result);
666 }
667
668 private static ClusterConnection createHConnectionCommon() {
669 ClusterConnection hc = Mockito.mock(ClusterConnection.class);
670 NonceGenerator ng = Mockito.mock(NonceGenerator.class);
671 Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE);
672 Mockito.when(hc.getNonceGenerator()).thenReturn(ng);
673 Mockito.when(hc.getConfiguration()).thenReturn(conf);
674 return hc;
675 }
676
677 @Test
678 public void testHTablePutSuccess() throws Exception {
679 BufferedMutatorImpl ht = Mockito.mock(BufferedMutatorImpl.class);
680 ht.ap = new MyAsyncProcess(createHConnection(), conf, true);
681
682 Put put = createPut(1, true);
683
684 Assert.assertEquals(0, ht.getWriteBufferSize());
685 ht.mutate(put);
686 Assert.assertEquals(0, ht.getWriteBufferSize());
687 }
688
689 private void doHTableFailedPut(boolean bufferOn) throws Exception {
690 ClusterConnection conn = createHConnection();
691 HTable ht = new HTable(conn, new BufferedMutatorParams(DUMMY_TABLE));
692 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
693 ht.mutator.ap = ap;
694 if (bufferOn) {
695 ht.setWriteBufferSize(1024L * 1024L);
696 } else {
697 ht.setWriteBufferSize(0L);
698 }
699
700 Put put = createPut(1, false);
701
702 Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize);
703 try {
704 ht.put(put);
705 if (bufferOn) {
706 ht.flushCommits();
707 }
708 Assert.fail();
709 } catch (RetriesExhaustedException expected) {
710 }
711 Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize);
712
713 AsyncRequestFuture ars = null;
714 for (AsyncRequestFuture someReqs : ap.allReqs) {
715 if (someReqs.getResults().length == 0) continue;
716 Assert.assertTrue(ars == null);
717 ars = someReqs;
718 }
719 Assert.assertTrue(ars != null);
720 verifyResult(ars, false);
721
722
723 ht.close();
724 }
725
726 @Test
727 public void testHTableFailedPutWithBuffer() throws Exception {
728 doHTableFailedPut(true);
729 }
730
731 @Test
732 public void testHTableFailedPutWithoutBuffer() throws Exception {
733 doHTableFailedPut(false);
734 }
735
736 @Test
737 public void testHTableFailedPutAndNewPut() throws Exception {
738 ClusterConnection conn = createHConnection();
739 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null,
740 new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(0));
741 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
742 mutator.ap = ap;
743
744 Put p = createPut(1, false);
745 mutator.mutate(p);
746
747 ap.waitUntilDone();
748
749
750
751
752
753
754 p = createPut(1, true);
755 Assert.assertEquals(0, mutator.getWriteBuffer().size());
756 try {
757 mutator.mutate(p);
758 Assert.fail();
759 } catch (RetriesExhaustedException expected) {
760 }
761 Assert.assertEquals("the put should not been inserted.", 0, mutator.getWriteBuffer().size());
762 }
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790 @Test
791 public void testBatch() throws IOException, InterruptedException {
792 ClusterConnection conn = new MyConnectionImpl(conf);
793 HTable ht = new HTable(conn, new BufferedMutatorParams(DUMMY_TABLE));
794 ht.multiAp = new MyAsyncProcess(conn, conf, false);
795
796 List<Put> puts = new ArrayList<Put>();
797 puts.add(createPut(1, true));
798 puts.add(createPut(1, true));
799 puts.add(createPut(1, true));
800 puts.add(createPut(1, true));
801 puts.add(createPut(1, false));
802 puts.add(createPut(1, true));
803 puts.add(createPut(1, false));
804
805 Object[] res = new Object[puts.size()];
806 try {
807 ht.processBatch(puts, res);
808 Assert.fail();
809 } catch (RetriesExhaustedException expected) {
810 }
811
812 Assert.assertEquals(res[0], success);
813 Assert.assertEquals(res[1], success);
814 Assert.assertEquals(res[2], success);
815 Assert.assertEquals(res[3], success);
816 Assert.assertEquals(res[4], failure);
817 Assert.assertEquals(res[5], success);
818 Assert.assertEquals(res[6], failure);
819 }
820
821 @Test
822 public void testErrorsServers() throws IOException {
823 Configuration configuration = new Configuration(conf);
824 ClusterConnection conn = new MyConnectionImpl(configuration);
825 BufferedMutatorImpl mutator =
826 new BufferedMutatorImpl(conn, null, null, new BufferedMutatorParams(DUMMY_TABLE));
827 configuration.setBoolean(ConnectionManager.RETRIES_BY_SERVER_KEY, true);
828
829 MyAsyncProcess ap = new MyAsyncProcess(conn, configuration, true);
830 mutator.ap = ap;
831
832 Assert.assertNotNull(mutator.ap.createServerErrorTracker());
833 Assert.assertTrue(mutator.ap.serverTrackerTimeout > 200);
834 mutator.ap.serverTrackerTimeout = 1;
835
836 Put p = createPut(1, false);
837 mutator.mutate(p);
838
839 try {
840 mutator.flush();
841 Assert.fail();
842 } catch (RetriesExhaustedWithDetailsException expected) {
843 }
844
845 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
846 }
847
848 @Test
849 public void testGlobalErrors() throws IOException {
850 ClusterConnection conn = new MyConnectionImpl(conf);
851 BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(DUMMY_TABLE);
852 AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, conf);
853 mutator.ap = ap;
854
855 Assert.assertNotNull(mutator.ap.createServerErrorTracker());
856
857 Put p = createPut(1, true);
858 mutator.mutate(p);
859
860 try {
861 mutator.flush();
862 Assert.fail();
863 } catch (RetriesExhaustedWithDetailsException expected) {
864 }
865
866 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
867 }
868
869
870
871
872
873 @Test
874 public void testThreadCreation() throws Exception {
875 final int NB_REGS = 100;
876 List<HRegionLocation> hrls = new ArrayList<HRegionLocation>(NB_REGS);
877 List<Get> gets = new ArrayList<Get>(NB_REGS);
878 for (int i = 0; i < NB_REGS; i++) {
879 HRegionInfo hri = new HRegionInfo(
880 DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i);
881 HRegionLocation hrl = new HRegionLocation(hri, i % 2 == 0 ? sn : sn2);
882 hrls.add(hrl);
883
884 Get get = new Get(Bytes.toBytes(i * 10L));
885 gets.add(get);
886 }
887
888 MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
889 HTable ht = new HTable(con, new BufferedMutatorParams(DUMMY_TABLE));
890 MyAsyncProcess ap = new MyAsyncProcess(con, conf, con.nbThreads);
891 ht.multiAp = ap;
892
893 ht.batch(gets, new Object[gets.size()]);
894
895 Assert.assertEquals(ap.nbActions.get(), NB_REGS);
896 Assert.assertEquals("1 multi response per server", 2, ap.nbMultiResponse.get());
897 Assert.assertEquals("1 thread per server", 2, con.nbThreads.get());
898
899 int nbReg = 0;
900 for (int i =0; i<NB_REGS; i++){
901 if (con.usedRegions[i]) nbReg++;
902 }
903 Assert.assertEquals("nbReg=" + nbReg, nbReg, NB_REGS);
904 }
905
906 @Test
907 public void testReplicaReplicaSuccess() throws Exception {
908
909
910 MyAsyncProcessWithReplicas ap = createReplicaAp(10, 1000, 0);
911 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3);
912 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[3]);
913 verifyReplicaResult(ars, RR.TRUE, RR.TRUE, RR.FALSE);
914 Assert.assertEquals(2, ap.getReplicaCallCount());
915 }
916
917 @Test
918 public void testReplicaPrimarySuccessWoReplicaCalls() throws Exception {
919
920 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 10, 0);
921 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3);
922 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[3]);
923 verifyReplicaResult(ars, RR.FALSE, RR.FALSE, RR.FALSE);
924 Assert.assertEquals(0, ap.getReplicaCallCount());
925 }
926
927 @Test
928 public void testReplicaParallelCallsSucceed() throws Exception {
929
930 MyAsyncProcessWithReplicas ap = createReplicaAp(0, 0, 0);
931 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
932 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
933 verifyReplicaResult(ars, RR.DONT_CARE, RR.DONT_CARE);
934 long replicaCalls = ap.getReplicaCallCount();
935 Assert.assertTrue(replicaCalls >= 0);
936 Assert.assertTrue(replicaCalls <= 2);
937 }
938
939 @Test
940 public void testReplicaPartialReplicaCall() throws Exception {
941
942
943
944 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0);
945 ap.setPrimaryCallDelay(sn2, 2000);
946 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
947 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
948 verifyReplicaResult(ars, RR.FALSE, RR.TRUE);
949 Assert.assertEquals(1, ap.getReplicaCallCount());
950 }
951
952 @Test
953 public void testReplicaMainFailsBeforeReplicaCalls() throws Exception {
954
955
956
957 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0, 1);
958 ap.addFailures(hri1, hri2);
959 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
960 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
961 verifyReplicaResult(ars, RR.FAILED, RR.FAILED);
962 Assert.assertEquals(0, ap.getReplicaCallCount());
963 }
964
965 @Test
966 public void testReplicaReplicaSuccessWithParallelFailures() throws Exception {
967
968
969 MyAsyncProcessWithReplicas ap = createReplicaAp(0, 1000, 1000, 1);
970 ap.addFailures(hri1, hri1r2, hri2);
971 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
972 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
973 verifyReplicaResult(ars, RR.TRUE, RR.TRUE);
974 Assert.assertEquals(2, ap.getReplicaCallCount());
975 }
976
977 @Test
978 public void testReplicaAllCallsFailForOneRegion() throws Exception {
979
980
981 MyAsyncProcessWithReplicas ap = createReplicaAp(500, 1000, 0, 1);
982 ap.addFailures(hri1, hri1r1, hri1r2, hri2r1);
983 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
984 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
985 verifyReplicaResult(ars, RR.FAILED, RR.FALSE);
986
987 Assert.assertEquals(3, ars.getErrors().getNumExceptions());
988 for (int i = 0; i < ars.getErrors().getNumExceptions(); ++i) {
989 Assert.assertArrayEquals(DUMMY_BYTES_1, ars.getErrors().getRow(i).getRow());
990 }
991 }
992
993 private MyAsyncProcessWithReplicas createReplicaAp(
994 int replicaAfterMs, int primaryMs, int replicaMs) throws Exception {
995 return createReplicaAp(replicaAfterMs, primaryMs, replicaMs, -1);
996 }
997
998 private MyAsyncProcessWithReplicas createReplicaAp(
999 int replicaAfterMs, int primaryMs, int replicaMs, int retries) throws Exception {
1000
1001
1002 Configuration conf = new Configuration();
1003 ClusterConnection conn = createHConnectionWithReplicas();
1004 conf.setInt(AsyncProcess.PRIMARY_CALL_TIMEOUT_KEY, replicaAfterMs * 1000);
1005 if (retries > 0) {
1006 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
1007 }
1008 MyAsyncProcessWithReplicas ap = new MyAsyncProcessWithReplicas(conn, conf);
1009 ap.setCallDelays(primaryMs, replicaMs);
1010 return ap;
1011 }
1012
1013 private static List<Get> makeTimelineGets(byte[]... rows) {
1014 List<Get> result = new ArrayList<Get>();
1015 for (byte[] row : rows) {
1016 Get get = new Get(row);
1017 get.setConsistency(Consistency.TIMELINE);
1018 result.add(get);
1019 }
1020 return result;
1021 }
1022
1023 private void verifyResult(AsyncRequestFuture ars, boolean... expected) throws Exception {
1024 Object[] actual = ars.getResults();
1025 Assert.assertEquals(expected.length, actual.length);
1026 for (int i = 0; i < expected.length; ++i) {
1027 Assert.assertEquals(expected[i], !(actual[i] instanceof Throwable));
1028 }
1029 }
1030
1031
1032 private enum RR {
1033 TRUE,
1034 FALSE,
1035 DONT_CARE,
1036 FAILED
1037 }
1038
1039 private void verifyReplicaResult(AsyncRequestFuture ars, RR... expecteds) throws Exception {
1040 Object[] actuals = ars.getResults();
1041 Assert.assertEquals(expecteds.length, actuals.length);
1042 for (int i = 0; i < expecteds.length; ++i) {
1043 Object actual = actuals[i];
1044 RR expected = expecteds[i];
1045 Assert.assertEquals(actual.toString(), expected == RR.FAILED, actual instanceof Throwable);
1046 if (expected != RR.FAILED && expected != RR.DONT_CARE) {
1047 Assert.assertEquals(expected == RR.TRUE, ((Result)actual).isStale());
1048 }
1049 }
1050 }
1051
1052
1053
1054
1055
1056
1057 private Put createPut(int regCnt, boolean success) {
1058 Put p;
1059 if (!success) {
1060 p = new Put(FAILS);
1061 } else switch (regCnt){
1062 case 1 :
1063 p = new Put(DUMMY_BYTES_1);
1064 break;
1065 case 2:
1066 p = new Put(DUMMY_BYTES_2);
1067 break;
1068 case 3:
1069 p = new Put(DUMMY_BYTES_3);
1070 break;
1071 default:
1072 throw new IllegalArgumentException("unknown " + regCnt);
1073 }
1074
1075 p.add(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
1076
1077 return p;
1078 }
1079
1080 @Test
1081 public void testWaitForMaximumCurrentTasks() throws Exception {
1082 final AtomicLong tasks = new AtomicLong(0);
1083 final AtomicInteger max = new AtomicInteger(0);
1084 final CyclicBarrier barrier = new CyclicBarrier(2);
1085 final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf);
1086 Runnable runnable = new Runnable() {
1087 @Override
1088 public void run() {
1089 try {
1090 barrier.await();
1091 ap.waitForMaximumCurrentTasks(max.get(), tasks, 1, null);
1092 } catch (InterruptedIOException e) {
1093 Assert.fail(e.getMessage());
1094 } catch (InterruptedException e) {
1095
1096 e.printStackTrace();
1097 } catch (BrokenBarrierException e) {
1098
1099 e.printStackTrace();
1100 }
1101 }
1102 };
1103
1104 Thread t = new Thread(runnable);
1105 t.start();
1106 barrier.await();
1107 t.join();
1108
1109 barrier.reset();
1110 tasks.set(1000000);
1111 t = new Thread(runnable);
1112 t.start();
1113 barrier.await();
1114 while (tasks.get() > 0) {
1115 assertTrue(t.isAlive());
1116 tasks.set(tasks.get() - 1);
1117 }
1118 t.join();
1119 }
1120 }