View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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   // Replica stuff
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       // Test HTable has tableName of null, so pass DUMMY_TABLE
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       // We use results in tests to check things, so override to always save them.
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             // sleep one second in order for threadpool to start another thread instead of reusing
226             // existing one.
227             Thread.sleep(1000);
228           } catch (InterruptedException e) {
229             // ignore error
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       // Currently AsyncProcess either sends all-replica, or all-primary request.
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    * Returns our async process.
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    * Returns our async process.
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); // 10 seconds max per method tested
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)); // <== this one won't be taken, the rs is busy
475     puts.add(createPut(1, true)); // <== this one will make it, the region is already in
476     puts.add(createPut(2, true)); // <== new region, but the rs is ok
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()); // TODO: this is timing-dependent
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     // Wait for AP to be free. While ars might have the result, ap counters are decreased later.
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     //Adds 100 to secure us against approximate timing.
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     // The table should have sent one request, maybe after multiple attempts
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     // This should not raise any exception, puts have been 'received' before by the catch.
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(); // Let's do all the retries.
748 
749     // We're testing that we're behaving as we were behaving in 0.94: sending exceptions in the
750     //  doPut if it fails.
751     // This said, it's not a very easy going behavior. For example, when we insert a list of
752     //  puts, we may raise an exception in the middle of the list. It's then up to the caller to
753     //  manage what was inserted, what was tried but failed, and what was not even tried.
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   @Test
767   public void testWithNoClearOnFail() throws IOException {
768     HTable ht = new HTable();
769     ht.ap = new MyAsyncProcess(createHConnection(), conf, true);
770     ht.setAutoFlushTo(false);
771 
772     Put p = createPut(1, false);
773     ht.put(p);
774     Assert.assertEquals(0, ht.writeAsyncBuffer.size());
775 
776     try {
777       ht.flushCommits();
778     } catch (RetriesExhaustedWithDetailsException expected) {
779     }
780     Assert.assertEquals(1, ht.writeAsyncBuffer.size());
781 
782     try {
783       ht.close();
784     } catch (RetriesExhaustedWithDetailsException expected) {
785     }
786     Assert.assertEquals(1, ht.writeAsyncBuffer.size());
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)); // <=== the bad apple, position 4
802     puts.add(createPut(1, true));
803     puts.add(createPut(1, false)); // <=== another bad apple, position 6
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     // Checking that the ErrorsServers came into play and didn't make us stop immediately
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     // Checking that the ErrorsServers came into play and didn't make us stop immediately
866     Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
867   }
868 
869   /**
870    * This test simulates multiple regions on 2 servers. We should have 2 multi requests and
871    *  2 threads: 1 per server, this whatever the number of regions.
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     // Main call takes too long so replicas succeed, except for one region w/o replicas.
909     // One region has no replica, so the main call succeeds for it.
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     // Main call succeeds before replica calls are kicked off.
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     // Either main or replica can succeed.
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     // One server is slow, so the result for its region comes from replica, whereas
942     // the result for other region comes from primary before replica calls happen.
943     // There should be no replica call for that region at all.
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     // Main calls fail before replica calls can start - this is currently not handled.
955     // It would probably never happen if we can get location (due to retries),
956     // and it would require additional synchronization.
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     // Main calls fails after replica calls start. For two-replica region, one replica call
968     // also fails. Regardless, we get replica results for both regions.
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     // For one of the region, all 3, main and replica, calls fail. For the other, replica
980     // call fails but its exception should not be visible as it did succeed.
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     // We should get 3 exceptions, for main + 2 replicas for DUMMY_BYTES_1
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     // TODO: this is kind of timing dependent... perhaps it should detect from createCaller
1001     //       that the replica call has happened and that way control the ordering.
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   /** After reading TheDailyWtf, I always wanted to create a MyBoolean enum like this! */
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    * @param regCnt  the region: 1 to 3.
1054    * @param success if true, the put will succeed.
1055    * @return a put
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           // TODO Auto-generated catch block
1096           e.printStackTrace();
1097         } catch (BrokenBarrierException e) {
1098           // TODO Auto-generated catch block
1099           e.printStackTrace();
1100         }
1101       }
1102     };
1103     // First test that our runnable thread only exits when tasks is zero.
1104     Thread t = new Thread(runnable);
1105     t.start();
1106     barrier.await();
1107     t.join();
1108     // Now assert we stay running if max == zero and tasks is > 0.
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 }