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