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 import static org.apache.hadoop.hbase.client.BufferedMutatorImpl.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS;
23 import static org.junit.Assert.assertEquals;
24 import static org.junit.Assert.assertFalse;
25 import static org.junit.Assert.assertNotEquals;
26 import static org.junit.Assert.assertTrue;
27 import static org.junit.Assert.fail;
28 import java.io.IOException;
29 import java.io.InterruptedIOException;
30 import java.util.ArrayList;
31 import java.util.Arrays;
32 import java.util.Collections;
33 import java.util.HashMap;
34 import java.util.Iterator;
35 import java.util.LinkedList;
36 import java.util.List;
37 import java.util.Map;
38 import java.util.Random;
39 import java.util.Set;
40 import java.util.TreeSet;
41 import java.util.concurrent.BlockingQueue;
42 import java.util.concurrent.BrokenBarrierException;
43 import java.util.concurrent.CyclicBarrier;
44 import java.util.concurrent.ExecutorService;
45 import java.util.concurrent.Future;
46 import java.util.concurrent.LinkedBlockingQueue;
47 import java.util.concurrent.RejectedExecutionException;
48 import java.util.concurrent.SynchronousQueue;
49 import java.util.concurrent.ThreadFactory;
50 import java.util.concurrent.ThreadPoolExecutor;
51 import java.util.concurrent.TimeUnit;
52 import java.util.concurrent.atomic.AtomicBoolean;
53 import java.util.concurrent.atomic.AtomicInteger;
54 import java.util.concurrent.atomic.AtomicLong;
55 import org.apache.commons.logging.Log;
56 import org.apache.commons.logging.LogFactory;
57 import org.apache.hadoop.conf.Configuration;
58 import org.apache.hadoop.hbase.CallQueueTooBigException;
59 import org.apache.hadoop.hbase.Cell;
60 import org.apache.hadoop.hbase.HConstants;
61 import org.apache.hadoop.hbase.HRegionInfo;
62 import org.apache.hadoop.hbase.HRegionLocation;
63 import org.apache.hadoop.hbase.RegionLocations;
64 import org.apache.hadoop.hbase.ServerName;
65 import org.apache.hadoop.hbase.TableName;
66 import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
67 import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFutureImpl;
68 import org.apache.hadoop.hbase.client.AsyncProcess.ListRowAccess;
69 import org.apache.hadoop.hbase.client.AsyncProcess.RequestSizeChecker;
70 import org.apache.hadoop.hbase.client.AsyncProcess.RowChecker;
71 import org.apache.hadoop.hbase.client.AsyncProcess.RowChecker.ReturnCode;
72 import org.apache.hadoop.hbase.client.AsyncProcess.RowCheckerHost;
73 import org.apache.hadoop.hbase.client.AsyncProcess.SubmittedSizeChecker;
74 import org.apache.hadoop.hbase.client.AsyncProcess.TaskCountChecker;
75 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
76 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
77 import org.apache.hadoop.hbase.client.coprocessor.Batch;
78 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
79 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
80 import org.apache.hadoop.hbase.testclassification.MediumTests;
81 import org.apache.hadoop.hbase.util.Bytes;
82 import org.apache.hadoop.hbase.util.Threads;
83 import org.junit.Assert;
84 import org.junit.BeforeClass;
85 import org.junit.Rule;
86 import org.junit.Test;
87 import org.junit.experimental.categories.Category;
88 import org.junit.rules.Timeout;
89 import org.mockito.Mockito;
90
91 @Category(MediumTests.class)
92 public class TestAsyncProcess {
93 private final static Log LOG = LogFactory.getLog(TestAsyncProcess.class);
94 private static final TableName DUMMY_TABLE =
95 TableName.valueOf("DUMMY_TABLE");
96 private static final byte[] DUMMY_BYTES_1 = Bytes.toBytes("DUMMY_BYTES_1");
97 private static final byte[] DUMMY_BYTES_2 = Bytes.toBytes("DUMMY_BYTES_2");
98 private static final byte[] DUMMY_BYTES_3 = Bytes.toBytes("DUMMY_BYTES_3");
99 private static final byte[] FAILS = Bytes.toBytes("FAILS");
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 private static int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
155 private long previousTimeout = -1;
156
157 @Override
158 <Res> AsyncRequestFutureImpl<Res> createAsyncRequestFuture(TableName tableName,
159 List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
160 Batch.Callback<Res> callback, Object[] results, boolean needResults,
161 PayloadCarryingServerCallable callable, int operationTimeout, int rpcTimeout) {
162
163 MyAsyncRequestFutureImpl<Res> r = new MyAsyncRequestFutureImpl(
164 DUMMY_TABLE, actions, nonceGroup, getPool(pool), needResults, results, callback, callable,
165 operationTimeout, rpcTimeout);
166 allReqs.add(r);
167 callsCt.incrementAndGet();
168 return r;
169 }
170
171 public MyAsyncProcess(ClusterConnection hc, Configuration conf) {
172 this(hc, conf, new AtomicInteger());
173 }
174
175 public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) {
176 super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
177 new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
178 new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf), rpcTimeout);
179 }
180
181 public MyAsyncProcess(
182 ClusterConnection hc, Configuration conf, boolean useGlobalErrors) {
183 super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
184 new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())),
185 new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf), rpcTimeout);
186 }
187
188 public MyAsyncProcess(ClusterConnection hc, Configuration conf, boolean useGlobalErrors,
189 @SuppressWarnings("unused") boolean dummy) {
190 super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
191 new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())) {
192 @Override
193 public void execute(Runnable command) {
194 throw new RejectedExecutionException("test under failure");
195 }
196 },
197 new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf), rpcTimeout);
198 }
199
200 @Override
201 public <Res> AsyncRequestFuture submit(TableName tableName, RowAccess<? extends Row> rows,
202 boolean atLeastOne, Callback<Res> callback, boolean needResults)
203 throws InterruptedIOException {
204
205 return super.submit(DUMMY_TABLE, rows, atLeastOne, callback, true);
206 }
207 @Override
208 public <Res> AsyncRequestFuture submit(TableName tableName, List<? extends Row> rows,
209 boolean atLeastOne, Callback<Res> callback, boolean needResults)
210 throws InterruptedIOException {
211
212 return super.submit(DUMMY_TABLE, rows, atLeastOne, callback, true);
213 }
214 @Override
215 public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
216 List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results,
217 PayloadCarryingServerCallable callable, int operationTimeout, int rpcTimeout) {
218 previousTimeout = rpcTimeout;
219 return super.submitAll(pool, tableName, rows, callback, results, callable, operationTimeout,
220 rpcTimeout);
221 }
222
223 @Override
224 protected void updateStats(ServerName server, Map<byte[], MultiResponse.RegionResult> results) {
225
226 }
227 @Override
228 protected RpcRetryingCaller<MultiResponse> createCaller(
229 PayloadCarryingServerCallable callable, int rpcTimeout) {
230 callsCt.incrementAndGet();
231 MultiServerCallable callable1 = (MultiServerCallable) callable;
232 final MultiResponse mr = createMultiResponse(
233 callable1.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() {
234 @Override
235 public void addResponse(MultiResponse mr, byte[] regionName, Action<Row> a) {
236 if (Arrays.equals(FAILS, a.getAction().getRow())) {
237 mr.add(regionName, a.getOriginalIndex(), failure);
238 } else {
239 mr.add(regionName, a.getOriginalIndex(), success);
240 }
241 }
242 });
243
244 return new RpcRetryingCaller<MultiResponse>(100, 500, 10, 9) {
245 @Override
246 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable,
247 int callTimeout)
248 throws IOException, RuntimeException {
249 try {
250
251
252 Thread.sleep(1000);
253 } catch (InterruptedException e) {
254
255 }
256 return mr;
257 }
258 };
259 }
260
261 class MyAsyncRequestFutureImpl<Res> extends AsyncRequestFutureImpl<Res> {
262
263 private final Map<ServerName, List<Long>> heapSizesByServer = new HashMap<>();
264
265 MyAsyncRequestFutureImpl(TableName tableName, List<Action<Row>> actions, long nonceGroup,
266 ExecutorService pool, boolean needResults, Object[] results,
267 Batch.Callback<Res> callback, PayloadCarryingServerCallable callable,
268 int operationTimeout, int rpcTimeout) {
269 super(tableName, actions, nonceGroup, pool, needResults, results, callback, callable, operationTimeout, rpcTimeout);
270 }
271
272 Map<ServerName, List<Long>> getRequestHeapSize() {
273 return heapSizesByServer;
274 }
275
276 @Override
277 SingleServerRequestRunnable createSingleServerRequest(
278 MultiAction<Row> multiAction, int numAttempt, ServerName server,
279 Set<PayloadCarryingServerCallable> callsInProgress) {
280 SingleServerRequestRunnable rq = new SingleServerRequestRunnable(
281 multiAction, numAttempt, server, callsInProgress);
282 List<Long> heapCount = heapSizesByServer.get(server);
283 if (heapCount == null) {
284 heapCount = new ArrayList<>();
285 heapSizesByServer.put(server, heapCount);
286 }
287 heapCount.add(heapSizeOf(multiAction));
288 return rq;
289 }
290
291 long heapSizeOf(MultiAction<Row> multiAction) {
292 long sum = 0;
293 for (List<Action<Row>> actions : multiAction.actions.values()) {
294 for (Action action : actions) {
295 Row row = action.getAction();
296 if (row instanceof Mutation) {
297 sum += ((Mutation) row).heapSize();
298 }
299 }
300 }
301 return sum;
302 }
303 }
304 }
305
306 static class CallerWithFailure extends RpcRetryingCaller<MultiResponse>{
307
308 private final IOException e;
309
310 public CallerWithFailure(IOException e) {
311 super(100, 500, 100, 9);
312 this.e = e;
313 }
314
315 @Override
316 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable,
317 int callTimeout)
318 throws IOException, RuntimeException {
319 throw e;
320 }
321 }
322
323
324 static class AsyncProcessWithFailure extends MyAsyncProcess {
325
326 private final IOException ioe;
327
328 public AsyncProcessWithFailure(ClusterConnection hc, Configuration conf, IOException ioe) {
329 super(hc, conf, true);
330 this.ioe = ioe;
331 serverTrackerTimeout = 1;
332 }
333
334 @Override
335 protected RpcRetryingCaller<MultiResponse> createCaller(
336 PayloadCarryingServerCallable callable, int rpcTimeout) {
337 callsCt.incrementAndGet();
338 return new CallerWithFailure(ioe);
339 }
340 }
341
342
343
344 static class MyClientBackoffPolicy implements ClientBackoffPolicy {
345 private final Map<ServerName, AtomicInteger> count = new HashMap<>();
346 @Override
347 public long getBackoffTime(ServerName serverName, byte[] region, ServerStatistics stats) {
348 AtomicInteger inc = count.get(serverName);
349 if (inc == null) {
350 inc = new AtomicInteger(0);
351 count.put(serverName, inc);
352 }
353 return inc.getAndIncrement();
354 }
355 }
356
357 static class MyAsyncProcessWithReplicas extends MyAsyncProcess {
358 private Set<byte[]> failures = new TreeSet<byte[]>(new Bytes.ByteArrayComparator());
359 private long primarySleepMs = 0, replicaSleepMs = 0;
360 private Map<ServerName, Long> customPrimarySleepMs = new HashMap<ServerName, Long>();
361 private final AtomicLong replicaCalls = new AtomicLong(0);
362
363 public void addFailures(HRegionInfo... hris) {
364 for (HRegionInfo hri : hris) {
365 failures.add(hri.getRegionName());
366 }
367 }
368
369 public long getReplicaCallCount() {
370 return replicaCalls.get();
371 }
372
373 public void setPrimaryCallDelay(ServerName server, long primaryMs) {
374 customPrimarySleepMs.put(server, primaryMs);
375 }
376
377 public MyAsyncProcessWithReplicas(ClusterConnection hc, Configuration conf) {
378 super(hc, conf);
379 }
380
381 public void setCallDelays(long primaryMs, long replicaMs) {
382 this.primarySleepMs = primaryMs;
383 this.replicaSleepMs = replicaMs;
384 }
385
386 @Override
387 protected RpcRetryingCaller<MultiResponse> createCaller(
388 PayloadCarryingServerCallable payloadCallable, int rpcTimeout) {
389 MultiServerCallable<Row> callable = (MultiServerCallable) payloadCallable;
390 final MultiResponse mr = createMultiResponse(
391 callable.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() {
392 @Override
393 public void addResponse(MultiResponse mr, byte[] regionName, Action<Row> a) {
394 if (failures.contains(regionName)) {
395 mr.add(regionName, a.getOriginalIndex(), failure);
396 } else {
397 boolean isStale = !RegionReplicaUtil.isDefaultReplica(a.getReplicaId());
398 mr.add(regionName, a.getOriginalIndex(),
399 Result.create(new Cell[0], null, isStale));
400 }
401 }
402 });
403
404 final boolean isDefault = RegionReplicaUtil.isDefaultReplica(
405 callable.getMulti().actions.values().iterator().next().iterator().next().getReplicaId());
406 final ServerName server = ((MultiServerCallable<?>)callable).getServerName();
407 String debugMsg = "Call to " + server + ", primary=" + isDefault + " with "
408 + callable.getMulti().actions.size() + " entries: ";
409 for (byte[] region : callable.getMulti().actions.keySet()) {
410 debugMsg += "[" + Bytes.toStringBinary(region) + "], ";
411 }
412 LOG.debug(debugMsg);
413 if (!isDefault) {
414 replicaCalls.incrementAndGet();
415 }
416
417 return new RpcRetryingCaller<MultiResponse>(100, 500, 10, 9) {
418 @Override
419 public MultiResponse callWithoutRetries(RetryingCallable<MultiResponse> callable,
420 int callTimeout)
421 throws IOException, RuntimeException {
422 long sleep = -1;
423 if (isDefault) {
424 Long customSleep = customPrimarySleepMs.get(server);
425 sleep = (customSleep == null ? primarySleepMs : customSleep.longValue());
426 } else {
427 sleep = replicaSleepMs;
428 }
429 if (sleep != 0) {
430 try {
431 Thread.sleep(sleep);
432 } catch (InterruptedException e) {
433 }
434 }
435 return mr;
436 }
437 };
438 }
439 }
440
441 static MultiResponse createMultiResponse(final MultiAction<Row> multi,
442 AtomicInteger nbMultiResponse, AtomicInteger nbActions, ResponseGenerator gen) {
443 final MultiResponse mr = new MultiResponse();
444 nbMultiResponse.incrementAndGet();
445 for (Map.Entry<byte[], List<Action<Row>>> entry : multi.actions.entrySet()) {
446 byte[] regionName = entry.getKey();
447 for (Action<Row> a : entry.getValue()) {
448 nbActions.incrementAndGet();
449 gen.addResponse(mr, regionName, a);
450 }
451 }
452 return mr;
453 }
454
455 private static interface ResponseGenerator {
456 void addResponse(final MultiResponse mr, byte[] regionName, Action<Row> a);
457 }
458
459
460
461
462 static class MyConnectionImpl extends ConnectionManager.HConnectionImplementation {
463 public static class TestConnectionRegistry implements ConnectionRegistry {
464 @Override
465 public void init(Connection connection) {}
466
467 @Override
468 public ServerName getActiveMaster() {
469 return null;
470 }
471
472 @Override
473 public RegionLocations getMetaRegionLocations() throws IOException {
474 return null;
475 }
476
477 @Override
478 public String getClusterId() {
479 return "testClusterId";
480 }
481
482 @Override
483 public boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException {
484 return false;
485 }
486
487 @Override
488 public int getCurrentNrHRS() throws IOException {
489 return 1;
490 }
491
492 @Override
493 public void close() {
494 }
495 }
496
497 final AtomicInteger nbThreads = new AtomicInteger(0);
498
499 protected MyConnectionImpl(Configuration conf) throws IOException {
500 super(setupConf(conf), false);
501 }
502
503 private static Configuration setupConf(Configuration conf) {
504 conf.setClass(HConstants.REGISTRY_IMPL_CONF_KEY, TestConnectionRegistry.class, ConnectionRegistry.class);
505 return conf;
506 }
507
508 @Override
509 public RegionLocations locateRegion(TableName tableName,
510 byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException {
511 return new RegionLocations(loc1);
512 }
513
514 @Override
515 public boolean hasCellBlockSupport() {
516 return false;
517 }
518 }
519
520
521
522
523 static class MyConnectionImpl2 extends MyConnectionImpl {
524 List<HRegionLocation> hrl;
525 final boolean usedRegions[];
526
527 protected MyConnectionImpl2(List<HRegionLocation> hrl) throws IOException {
528 super(conf);
529 this.hrl = hrl;
530 this.usedRegions = new boolean[hrl.size()];
531 }
532
533 @Override
534 public RegionLocations locateRegion(TableName tableName,
535 byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException {
536 int i = 0;
537 for (HRegionLocation hr : hrl){
538 if (Arrays.equals(row, hr.getRegionInfo().getStartKey())) {
539 usedRegions[i] = true;
540 return new RegionLocations(hr);
541 }
542 i++;
543 }
544 return null;
545 }
546 }
547
548 @Rule
549 public Timeout timeout = Timeout.millis(10000);
550
551 @Test
552 public void testSubmit() throws Exception {
553 ClusterConnection hc = createHConnection();
554 AsyncProcess ap = new MyAsyncProcess(hc, conf);
555
556 List<Put> puts = new ArrayList<Put>();
557 puts.add(createPut(1, true));
558
559 ap.submit(DUMMY_TABLE, puts, false, null, false);
560 Assert.assertTrue(puts.isEmpty());
561 }
562 @Test
563 public void testListRowAccess() {
564 int count = 10;
565 List<String> values = new LinkedList<>();
566 for (int i = 0; i != count; ++i) {
567 values.add(String.valueOf(i));
568 }
569
570 ListRowAccess<String> taker = new ListRowAccess(values);
571 assertEquals(count, taker.size());
572
573 int restoreCount = 0;
574 int takeCount = 0;
575 Iterator<String> it = taker.iterator();
576 while (it.hasNext()) {
577 String v = it.next();
578 assertEquals(String.valueOf(takeCount), v);
579 ++takeCount;
580 it.remove();
581 if (Math.random() >= 0.5) {
582 break;
583 }
584 }
585 assertEquals(count, taker.size() + takeCount);
586
587 it = taker.iterator();
588 while (it.hasNext()) {
589 String v = it.next();
590 assertEquals(String.valueOf(takeCount), v);
591 ++takeCount;
592 it.remove();
593 }
594 assertEquals(0, taker.size());
595 assertEquals(count, takeCount);
596 }
597 private static long calculateRequestCount(long putSizePerServer, long maxHeapSizePerRequest) {
598 if (putSizePerServer <= maxHeapSizePerRequest) {
599 return 1;
600 } else if (putSizePerServer % maxHeapSizePerRequest == 0) {
601 return putSizePerServer / maxHeapSizePerRequest;
602 } else {
603 return putSizePerServer / maxHeapSizePerRequest + 1;
604 }
605 }
606
607 @Test
608 public void testSubmitSameSizeOfRequest() throws Exception {
609 long writeBuffer = 2 * 1024 * 1024;
610 long putsHeapSize = writeBuffer;
611 doSubmitRequest(writeBuffer, putsHeapSize);
612 }
613 @Test
614 public void testIllegalArgument() throws IOException {
615 ClusterConnection conn = createHConnection();
616 final long maxHeapSizePerRequest = conn.getConfiguration().getLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
617 AsyncProcess.DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE);
618 conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, -1);
619 try {
620 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
621 fail("The maxHeapSizePerRequest must be bigger than zero");
622 } catch (IllegalArgumentException e) {
623 }
624 conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, maxHeapSizePerRequest);
625 }
626 @Test
627 public void testSubmitLargeRequestWithUnlimitedSize() throws Exception {
628 long maxHeapSizePerRequest = Long.MAX_VALUE;
629 long putsHeapSize = 2 * 1024 * 1024;
630 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize);
631 }
632
633 @Test(timeout=300000)
634 public void testSubmitRandomSizeRequest() throws Exception {
635 Random rn = new Random();
636 final long limit = 10 * 1024 * 1024;
637 final int requestCount = 1 + (int) (rn.nextDouble() * 3);
638 long n = rn.nextLong();
639 if (n < 0) {
640 n = -n;
641 } else if (n == 0) {
642 n = 1;
643 }
644 long putsHeapSize = n % limit;
645 long maxHeapSizePerRequest = putsHeapSize / requestCount;
646 LOG.info("[testSubmitRandomSizeRequest] maxHeapSizePerRequest=" + maxHeapSizePerRequest +
647 ", putsHeapSize=" + putsHeapSize);
648 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize);
649 }
650
651 @Test
652 public void testSubmitSmallRequest() throws Exception {
653 long maxHeapSizePerRequest = 2 * 1024 * 1024;
654 long putsHeapSize = 100;
655 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize);
656 }
657
658 @Test(timeout=120000)
659 public void testSubmitLargeRequest() throws Exception {
660 long maxHeapSizePerRequest = 2 * 1024 * 1024;
661 long putsHeapSize = maxHeapSizePerRequest * 2;
662 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize);
663 }
664
665 private void doSubmitRequest(long maxHeapSizePerRequest, long putsHeapSize) throws Exception {
666 ClusterConnection conn = createHConnection();
667 final long defaultHeapSizePerRequest = conn.getConfiguration().getLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
668 AsyncProcess.DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE);
669 conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, maxHeapSizePerRequest);
670 BufferedMutatorParams bufferParam = new BufferedMutatorParams(DUMMY_TABLE);
671
672
673 long putSizeSN = 0;
674 long putSizeSN2 = 0;
675 List<Put> puts = new ArrayList<>();
676 while ((putSizeSN + putSizeSN2) <= putsHeapSize) {
677 Put put1 = new Put(DUMMY_BYTES_1);
678 put1.addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
679 Put put2 = new Put(DUMMY_BYTES_2);
680 put2.addColumn(DUMMY_BYTES_2, DUMMY_BYTES_2, DUMMY_BYTES_2);
681 Put put3 = new Put(DUMMY_BYTES_3);
682 put3.addColumn(DUMMY_BYTES_3, DUMMY_BYTES_3, DUMMY_BYTES_3);
683 putSizeSN += (put1.heapSize() + put2.heapSize());
684 putSizeSN2 += put3.heapSize();
685 puts.add(put1);
686 puts.add(put2);
687 puts.add(put3);
688 }
689
690 int minCountSnRequest = (int) calculateRequestCount(putSizeSN, maxHeapSizePerRequest);
691 int minCountSn2Request = (int) calculateRequestCount(putSizeSN2, maxHeapSizePerRequest);
692 LOG.info("Total put count:" + puts.size() + ", putSizeSN:"+ putSizeSN + ", putSizeSN2:" + putSizeSN2
693 + ", maxHeapSizePerRequest:" + maxHeapSizePerRequest
694 + ", minCountSnRequest:" + minCountSnRequest
695 + ", minCountSn2Request:" + minCountSn2Request);
696 try (HTable ht = new HTable(conn, bufferParam)) {
697 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
698 ht.mutator.ap = ap;
699
700 Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get());
701 ht.put(puts);
702 List<AsyncRequestFuture> reqs = ap.allReqs;
703
704 int actualSnReqCount = 0;
705 int actualSn2ReqCount = 0;
706 for (AsyncRequestFuture req : reqs) {
707 if (!(req instanceof AsyncRequestFutureImpl)) {
708 continue;
709 }
710 MyAsyncProcess.MyAsyncRequestFutureImpl ars = (MyAsyncProcess.MyAsyncRequestFutureImpl) req;
711 if (ars.getRequestHeapSize().containsKey(sn)) {
712 ++actualSnReqCount;
713 }
714 if (ars.getRequestHeapSize().containsKey(sn2)) {
715 ++actualSn2ReqCount;
716 }
717 }
718
719 assertEquals(true, minCountSnRequest <= actualSnReqCount);
720 assertEquals(true, minCountSn2Request <= actualSn2ReqCount);
721 Map<ServerName, Long> sizePerServers = new HashMap<>();
722 for (AsyncRequestFuture req : reqs) {
723 if (!(req instanceof AsyncRequestFutureImpl)) {
724 continue;
725 }
726 MyAsyncProcess.MyAsyncRequestFutureImpl ars = (MyAsyncProcess.MyAsyncRequestFutureImpl) req;
727 Map<ServerName, List<Long>> requestHeapSize = ars.getRequestHeapSize();
728 for (Map.Entry<ServerName, List<Long>> entry : requestHeapSize.entrySet()) {
729 long sum = 0;
730 for (long size : entry.getValue()) {
731 assertEquals(true, size <= maxHeapSizePerRequest);
732 sum += size;
733 }
734 assertEquals(true, sum <= maxHeapSizePerRequest);
735 long value = sizePerServers.containsKey(entry.getKey()) ? sizePerServers.get(entry.getKey()) : 0L;
736 sizePerServers.put(entry.getKey(), value + sum);
737 }
738 }
739 assertEquals(true, sizePerServers.containsKey(sn));
740 assertEquals(true, sizePerServers.containsKey(sn2));
741 assertEquals(false, sizePerServers.containsKey(sn3));
742 assertEquals(putSizeSN, (long) sizePerServers.get(sn));
743 assertEquals(putSizeSN2, (long) sizePerServers.get(sn2));
744 }
745
746 conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, defaultHeapSizePerRequest);
747 }
748 @Test
749 public void testSubmitWithCB() throws Exception {
750 ClusterConnection hc = createHConnection();
751 final AtomicInteger updateCalled = new AtomicInteger(0);
752 Batch.Callback<Object> cb = new Batch.Callback<Object>() {
753 @Override
754 public void update(byte[] region, byte[] row, Object result) {
755 updateCalled.incrementAndGet();
756 }
757 };
758 AsyncProcess ap = new MyAsyncProcess(hc, conf);
759
760 List<Put> puts = new ArrayList<Put>();
761 puts.add(createPut(1, true));
762
763 final AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, cb, false);
764 Assert.assertTrue(puts.isEmpty());
765 ars.waitUntilDone();
766 Assert.assertEquals(1, updateCalled.get());
767 }
768
769 @Test
770 public void testSubmitBusyRegion() throws Exception {
771 ClusterConnection hc = createHConnection();
772 AsyncProcess ap = new MyAsyncProcess(hc, conf);
773
774 List<Put> puts = new ArrayList<Put>();
775 puts.add(createPut(1, true));
776
777 for (int i = 0; i != ap.maxConcurrentTasksPerRegion; ++i) {
778 ap.incTaskCounters(Collections.singletonList(hri1.getRegionName()), sn);
779 }
780 ap.submit(DUMMY_TABLE, puts, false, null, false);
781 Assert.assertEquals(puts.size(), 1);
782
783 ap.decTaskCounters(Collections.singletonList(hri1.getRegionName()), sn);
784 ap.submit(DUMMY_TABLE, puts, false, null, false);
785 Assert.assertEquals(0, puts.size());
786 }
787
788
789 @Test
790 public void testSubmitBusyRegionServer() throws Exception {
791 ClusterConnection hc = createHConnection();
792 AsyncProcess ap = new MyAsyncProcess(hc, conf);
793
794 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer));
795
796 List<Put> puts = new ArrayList<Put>();
797 puts.add(createPut(1, true));
798 puts.add(createPut(3, true));
799 puts.add(createPut(1, true));
800 puts.add(createPut(2, true));
801
802 ap.submit(DUMMY_TABLE, puts, false, null, false);
803 Assert.assertEquals(" puts=" + puts, 1, puts.size());
804
805 ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer - 1));
806 ap.submit(DUMMY_TABLE, puts, false, null, false);
807 Assert.assertTrue(puts.isEmpty());
808 }
809
810 @Test
811 public void testFail() throws Exception {
812 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
813
814 List<Put> puts = new ArrayList<Put>();
815 Put p = createPut(1, false);
816 puts.add(p);
817
818 AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
819 Assert.assertEquals(0, puts.size());
820 ars.waitUntilDone();
821 verifyResult(ars, false);
822 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
823
824 Assert.assertEquals(1, ars.getErrors().exceptions.size());
825 Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0),
826 failure.equals(ars.getErrors().exceptions.get(0)));
827 Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0),
828 failure.equals(ars.getErrors().exceptions.get(0)));
829
830 Assert.assertEquals(1, ars.getFailedOperations().size());
831 Assert.assertTrue("was: " + ars.getFailedOperations().get(0),
832 p.equals(ars.getFailedOperations().get(0)));
833 }
834
835
836 @Test
837 public void testSubmitTrue() throws IOException {
838 final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
839 ap.tasksInProgress.incrementAndGet();
840 final AtomicInteger ai = new AtomicInteger(ap.maxConcurrentTasksPerRegion);
841 ap.taskCounterPerRegion.put(hri1.getRegionName(), ai);
842
843 final AtomicBoolean checkPoint = new AtomicBoolean(false);
844 final AtomicBoolean checkPoint2 = new AtomicBoolean(false);
845
846 Thread t = new Thread(){
847 @Override
848 public void run(){
849 Threads.sleep(1000);
850 assertFalse(checkPoint.get());
851 ai.decrementAndGet();
852 ap.tasksInProgress.decrementAndGet();
853 checkPoint2.set(true);
854 }
855 };
856
857 List<Put> puts = new ArrayList<Put>();
858 Put p = createPut(1, true);
859 puts.add(p);
860
861 ap.submit(DUMMY_TABLE, puts, false, null, false);
862 assertFalse(puts.isEmpty());
863
864 t.start();
865
866 ap.submit(DUMMY_TABLE, puts, true, null, false);
867 Assert.assertTrue(puts.isEmpty());
868
869 checkPoint.set(true);
870 while (!checkPoint2.get()){
871 Threads.sleep(1);
872 }
873 }
874
875 @Test
876 public void testFailAndSuccess() throws Exception {
877 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
878
879 List<Put> puts = new ArrayList<Put>();
880 puts.add(createPut(1, false));
881 puts.add(createPut(1, true));
882 puts.add(createPut(1, true));
883
884 AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
885 Assert.assertTrue(puts.isEmpty());
886 ars.waitUntilDone();
887 verifyResult(ars, false, true, true);
888 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
889 ap.callsCt.set(0);
890 Assert.assertEquals(1, ars.getErrors().actions.size());
891
892 puts.add(createPut(1, true));
893
894 ap.waitUntilDone();
895 ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
896 Assert.assertEquals(0, puts.size());
897 ars.waitUntilDone();
898 Assert.assertEquals(2, ap.callsCt.get());
899 verifyResult(ars, true);
900 }
901
902 @Test
903 public void testFlush() throws Exception {
904 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
905
906 List<Put> puts = new ArrayList<Put>();
907 puts.add(createPut(1, false));
908 puts.add(createPut(1, true));
909 puts.add(createPut(1, true));
910
911 AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
912 ars.waitUntilDone();
913 verifyResult(ars, false, true, true);
914 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
915
916 Assert.assertEquals(1, ars.getFailedOperations().size());
917 }
918
919 @Test
920 public void testTaskCountWithoutClientBackoffPolicy() throws IOException, InterruptedException {
921 ClusterConnection hc = createHConnection();
922 MyAsyncProcess ap = new MyAsyncProcess(hc, conf, false);
923 testTaskCount(ap);
924 }
925
926 @Test
927 public void testTaskCountWithClientBackoffPolicy() throws IOException, InterruptedException {
928 Configuration copyConf = new Configuration(conf);
929 copyConf.setBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, true);
930 MyClientBackoffPolicy bp = new MyClientBackoffPolicy();
931 ClusterConnection hc = createHConnection();
932 Mockito.when(hc.getConfiguration()).thenReturn(copyConf);
933 Mockito.when(hc.getStatisticsTracker()).thenReturn(ServerStatisticTracker.create(copyConf));
934 Mockito.when(hc.getBackoffPolicy()).thenReturn(bp);
935 MyAsyncProcess ap = new MyAsyncProcess(hc, copyConf, false);
936 testTaskCount(ap);
937 }
938
939 private void testTaskCount(AsyncProcess ap) throws InterruptedIOException, InterruptedException {
940 List<Put> puts = new ArrayList<>();
941 for (int i = 0; i != 3; ++i) {
942 puts.add(createPut(1, true));
943 puts.add(createPut(2, true));
944 puts.add(createPut(3, true));
945 }
946 ap.submit(DUMMY_TABLE, puts, true, null, false);
947 ap.waitUntilDone();
948
949 TimeUnit.SECONDS.sleep(1);
950 assertEquals(0, ap.tasksInProgress.get());
951 for (AtomicInteger count : ap.taskCounterPerRegion.values()) {
952 assertEquals(0, count.get());
953 }
954 for (AtomicInteger count : ap.taskCounterPerServer.values()) {
955 assertEquals(0, count.get());
956 }
957 }
958
959 @Test
960 public void testMaxTask() throws Exception {
961 final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
962
963 for (int i = 0; i < 1000; i++) {
964 ap.incTaskCounters(Collections.singletonList(Bytes.toBytes("dummy")), sn);
965 }
966
967 final Thread myThread = Thread.currentThread();
968
969 Thread t = new Thread() {
970 @Override
971 public void run() {
972 Threads.sleep(2000);
973 myThread.interrupt();
974 }
975 };
976
977 List<Put> puts = new ArrayList<Put>();
978 puts.add(createPut(1, true));
979
980 t.start();
981
982 try {
983 ap.submit(DUMMY_TABLE, puts, false, null, false);
984 Assert.fail("We should have been interrupted.");
985 } catch (InterruptedIOException expected) {
986 }
987
988 final long sleepTime = 2000;
989
990 Thread t2 = new Thread() {
991 @Override
992 public void run() {
993 Threads.sleep(sleepTime);
994 while (ap.tasksInProgress.get() > 0) {
995 ap.decTaskCounters(Collections.singletonList(Bytes.toBytes("dummy")), sn);
996 }
997 }
998 };
999 t2.start();
1000
1001 long start = System.currentTimeMillis();
1002 ap.submit(DUMMY_TABLE, new ArrayList<Row>(), false, null, false);
1003 long end = System.currentTimeMillis();
1004
1005
1006 Assert.assertTrue(start + 100L + sleepTime > end);
1007 }
1008
1009 private static ClusterConnection createHConnection() throws IOException {
1010 ClusterConnection hc = createHConnectionCommon();
1011 setMockLocation(hc, DUMMY_BYTES_1, new RegionLocations(loc1));
1012 setMockLocation(hc, DUMMY_BYTES_2, new RegionLocations(loc2));
1013 setMockLocation(hc, DUMMY_BYTES_3, new RegionLocations(loc3));
1014 Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), Mockito.anyBoolean()))
1015 .thenReturn(Arrays.asList(loc1, loc2, loc3));
1016 setMockLocation(hc, FAILS, new RegionLocations(loc2));
1017 return hc;
1018 }
1019
1020 private static ClusterConnection createHConnectionWithReplicas() throws IOException {
1021 ClusterConnection hc = createHConnectionCommon();
1022 setMockLocation(hc, DUMMY_BYTES_1, hrls1);
1023 setMockLocation(hc, DUMMY_BYTES_2, hrls2);
1024 setMockLocation(hc, DUMMY_BYTES_3, hrls3);
1025 List<HRegionLocation> locations = new ArrayList<HRegionLocation>();
1026 for (HRegionLocation loc : hrls1.getRegionLocations()) {
1027 locations.add(loc);
1028 }
1029 for (HRegionLocation loc : hrls2.getRegionLocations()) {
1030 locations.add(loc);
1031 }
1032 for (HRegionLocation loc : hrls3.getRegionLocations()) {
1033 locations.add(loc);
1034 }
1035 Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), Mockito.anyBoolean()))
1036 .thenReturn(locations);
1037 return hc;
1038 }
1039
1040 private static void setMockLocation(ClusterConnection hc, byte[] row,
1041 RegionLocations result) throws IOException {
1042 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row),
1043 Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result);
1044 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row),
1045 Mockito.anyBoolean(), Mockito.anyBoolean())).thenReturn(result);
1046 }
1047
1048 private static ClusterConnection createHConnectionCommon() {
1049 ClusterConnection hc = Mockito.mock(ClusterConnection.class);
1050 NonceGenerator ng = Mockito.mock(NonceGenerator.class);
1051 Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE);
1052 Mockito.when(hc.getNonceGenerator()).thenReturn(ng);
1053 Mockito.when(hc.getConfiguration()).thenReturn(conf);
1054 return hc;
1055 }
1056
1057 @Test
1058 public void testHTablePutSuccess() throws Exception {
1059 BufferedMutatorImpl ht = Mockito.mock(BufferedMutatorImpl.class);
1060 ht.ap = new MyAsyncProcess(createHConnection(), conf, true);
1061
1062 Put put = createPut(1, true);
1063
1064 Assert.assertEquals(0, ht.getWriteBufferSize());
1065 ht.mutate(put);
1066 Assert.assertEquals(0, ht.getWriteBufferSize());
1067 }
1068
1069 private void doHTableFailedPut(boolean bufferOn) throws Exception {
1070 ClusterConnection conn = createHConnection();
1071 HTable ht = new HTable(conn, new BufferedMutatorParams(DUMMY_TABLE));
1072 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
1073 ht.mutator.ap = ap;
1074 if (bufferOn) {
1075 ht.setWriteBufferSize(1024L * 1024L);
1076 } else {
1077 ht.setWriteBufferSize(0L);
1078 }
1079
1080 Put put = createPut(1, false);
1081
1082 Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get());
1083 try {
1084 ht.put(put);
1085 if (bufferOn) {
1086 ht.flushCommits();
1087 }
1088 Assert.fail();
1089 } catch (RetriesExhaustedException expected) {
1090 }
1091 Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get());
1092
1093 AsyncRequestFuture ars = null;
1094 for (AsyncRequestFuture someReqs : ap.allReqs) {
1095 if (someReqs.getResults().length == 0) continue;
1096 Assert.assertTrue(ars == null);
1097 ars = someReqs;
1098 }
1099 Assert.assertTrue(ars != null);
1100 verifyResult(ars, false);
1101
1102
1103 ht.close();
1104 }
1105
1106 @Test
1107 public void testHTableFailedPutWithBuffer() throws Exception {
1108 doHTableFailedPut(true);
1109 }
1110
1111 @Test
1112 public void testHTableFailedPutWithoutBuffer() throws Exception {
1113 doHTableFailedPut(false);
1114 }
1115
1116 @Test
1117 public void testHTableFailedPutAndNewPut() throws Exception {
1118 ClusterConnection conn = createHConnection();
1119 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null,
1120 new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(0));
1121 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
1122 mutator.ap = ap;
1123
1124 Put p = createPut(1, false);
1125 mutator.mutate(p);
1126
1127 ap.waitUntilDone();
1128
1129
1130
1131
1132
1133
1134 p = createPut(1, true);
1135 Assert.assertEquals(0, mutator.getWriteBuffer().size());
1136 try {
1137 mutator.mutate(p);
1138 Assert.fail();
1139 } catch (RetriesExhaustedException expected) {
1140 }
1141 Assert.assertEquals("the put should not been inserted.", 0, mutator.getWriteBuffer().size());
1142 }
1143
1144 @Test
1145 public void testSettingWriteBufferPeriodicFlushParameters() throws Exception {
1146 ClusterConnection conn = createHConnection();
1147 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
1148
1149 checkPeriodicFlushParameters(conn, ap,
1150 1234, 1234,
1151 1234, 1234);
1152 checkPeriodicFlushParameters(conn, ap,
1153 0, 0,
1154 0, MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS);
1155 checkPeriodicFlushParameters(conn, ap,
1156 -1234, 0,
1157 -1234, MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS);
1158 checkPeriodicFlushParameters(conn, ap,
1159 1, 1,
1160 1, MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS);
1161 }
1162
1163 private void checkPeriodicFlushParameters(ClusterConnection conn,
1164 MyAsyncProcess ap,
1165 long setTO, long expectTO,
1166 long setTT, long expectTT
1167 ) {
1168 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
1169
1170
1171 bufferParam.setWriteBufferPeriodicFlushTimeoutMs(setTO);
1172 bufferParam.setWriteBufferPeriodicFlushTimerTickMs(setTT);
1173 Assert.assertEquals(setTO, bufferParam.getWriteBufferPeriodicFlushTimeoutMs());
1174 Assert.assertEquals(setTT, bufferParam.getWriteBufferPeriodicFlushTimerTickMs());
1175
1176
1177 BufferedMutatorImpl ht1 = new BufferedMutatorImpl(conn, null, null, bufferParam);
1178 Assert.assertEquals(expectTO, ht1.getWriteBufferPeriodicFlushTimeoutMs());
1179 Assert.assertEquals(expectTT, ht1.getWriteBufferPeriodicFlushTimerTickMs());
1180
1181
1182 BufferedMutatorImpl ht2 =
1183 new BufferedMutatorImpl(conn, null, null, createBufferedMutatorParams(ap, DUMMY_TABLE));
1184 ht2.setWriteBufferPeriodicFlush(setTO, setTT);
1185 Assert.assertEquals(expectTO, ht2.getWriteBufferPeriodicFlushTimeoutMs());
1186 Assert.assertEquals(expectTT, ht2.getWriteBufferPeriodicFlushTimerTickMs());
1187
1188 }
1189
1190 @Test
1191 public void testWriteBufferPeriodicFlushTimeoutMs() throws Exception {
1192 ClusterConnection conn = createHConnection();
1193 MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
1194 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
1195
1196 bufferParam.setWriteBufferPeriodicFlushTimeoutMs(1);
1197 bufferParam.setWriteBufferPeriodicFlushTimerTickMs(1);
1198 bufferParam.writeBufferSize(10000);
1199
1200 BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, null, null, bufferParam);
1201 ht.ap = ap;
1202
1203
1204 Assert.assertEquals(10000, ht.getWriteBufferSize());
1205 Assert.assertEquals(1, ht.getWriteBufferPeriodicFlushTimeoutMs());
1206 Assert.assertEquals(MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS,
1207 ht.getWriteBufferPeriodicFlushTimerTickMs());
1208
1209 Put put = createPut(1, true);
1210
1211 Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes());
1212 Assert.assertEquals(0, ht.getCurrentWriteBufferSize());
1213
1214
1215 ht.mutate(put);
1216 ht.flush();
1217
1218 Thread.sleep(1000);
1219 Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes());
1220 Assert.assertEquals(0, ht.getCurrentWriteBufferSize());
1221
1222
1223 ht.mutate(put);
1224 Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes());
1225 Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0);
1226
1227
1228
1229 Thread.sleep(200);
1230 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes());
1231 Assert.assertEquals(0, ht.getCurrentWriteBufferSize());
1232
1233
1234 Thread.sleep(200);
1235 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes());
1236 Assert.assertEquals(0, ht.getCurrentWriteBufferSize());
1237
1238
1239 ht.disableWriteBufferPeriodicFlush();
1240 ht.mutate(put);
1241 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes());
1242 Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0);
1243
1244
1245 Thread.sleep(200);
1246 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes());
1247 Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0);
1248
1249
1250
1251 ht.setWriteBufferPeriodicFlush(1, 100);
1252 Thread.sleep(2000);
1253 Assert.assertEquals(2, ht.getExecutedWriteBufferPeriodicFlushes());
1254 Assert.assertEquals(0, ht.getCurrentWriteBufferSize());
1255 }
1256
1257 @Test
1258 public void testTaskCheckerHost() throws IOException {
1259 final int maxTotalConcurrentTasks = 100;
1260 final int maxConcurrentTasksPerServer = 2;
1261 final int maxConcurrentTasksPerRegion = 1;
1262 final AtomicLong tasksInProgress = new AtomicLong(0);
1263 final Map<ServerName, AtomicInteger> taskCounterPerServer = new HashMap<>();
1264 final Map<byte[], AtomicInteger> taskCounterPerRegion = new HashMap<>();
1265 TaskCountChecker countChecker = new TaskCountChecker(
1266 maxTotalConcurrentTasks,
1267 maxConcurrentTasksPerServer,
1268 maxConcurrentTasksPerRegion,
1269 tasksInProgress, taskCounterPerServer, taskCounterPerRegion);
1270 final long maxHeapSizePerRequest = 2 * 1024 * 1024;
1271
1272 RequestSizeChecker sizeChecker = new RequestSizeChecker(maxHeapSizePerRequest);
1273 RowCheckerHost checkerHost = new RowCheckerHost(Arrays.asList(countChecker, sizeChecker));
1274
1275 ReturnCode loc1Code = checkerHost.canTakeOperation(loc1, maxHeapSizePerRequest);
1276 assertEquals(RowChecker.ReturnCode.INCLUDE, loc1Code);
1277
1278 ReturnCode loc1Code_2 = checkerHost.canTakeOperation(loc1, maxHeapSizePerRequest);
1279
1280 assertNotEquals(RowChecker.ReturnCode.INCLUDE, loc1Code_2);
1281
1282 ReturnCode loc2Code = checkerHost.canTakeOperation(loc2, maxHeapSizePerRequest);
1283
1284 assertNotEquals(RowChecker.ReturnCode.INCLUDE, loc2Code);
1285
1286
1287 taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(100));
1288 taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(100));
1289
1290 ReturnCode loc3Code = checkerHost.canTakeOperation(loc3, 1L);
1291
1292 assertNotEquals(RowChecker.ReturnCode.INCLUDE, loc3Code);
1293
1294
1295 taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(0));
1296 taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(0));
1297
1298 ReturnCode loc3Code_2 = checkerHost.canTakeOperation(loc3, 1L);
1299 assertEquals(RowChecker.ReturnCode.INCLUDE, loc3Code_2);
1300 }
1301
1302 @Test
1303 public void testRequestSizeCheckerr() throws IOException {
1304 final long maxHeapSizePerRequest = 2 * 1024 * 1024;
1305 final ClusterConnection conn = createHConnection();
1306 RequestSizeChecker checker = new RequestSizeChecker(maxHeapSizePerRequest);
1307
1308
1309 for (int i = 0; i != 10; ++i) {
1310 ReturnCode code = checker.canTakeOperation(loc1, maxHeapSizePerRequest);
1311 assertEquals(RowChecker.ReturnCode.INCLUDE, code);
1312 code = checker.canTakeOperation(loc2, maxHeapSizePerRequest);
1313 assertEquals(RowChecker.ReturnCode.INCLUDE, code);
1314 }
1315
1316
1317 ReturnCode acceptCode = checker.canTakeOperation(loc1, maxHeapSizePerRequest);
1318 assertEquals(RowChecker.ReturnCode.INCLUDE, acceptCode);
1319 checker.notifyFinal(acceptCode, loc1, maxHeapSizePerRequest);
1320
1321
1322 for (int i = 0; i != 10; ++i) {
1323 ReturnCode code = checker.canTakeOperation(loc1, maxHeapSizePerRequest);
1324 assertNotEquals(RowChecker.ReturnCode.INCLUDE, code);
1325 code = checker.canTakeOperation(loc2, maxHeapSizePerRequest);
1326 assertNotEquals(RowChecker.ReturnCode.INCLUDE, code);
1327 }
1328
1329
1330 for (int i = 0; i != 10; ++i) {
1331 ReturnCode code = checker.canTakeOperation(loc3, maxHeapSizePerRequest);
1332 assertEquals(RowChecker.ReturnCode.INCLUDE, code);
1333 }
1334
1335 checker.reset();
1336 for (int i = 0; i != 10; ++i) {
1337 ReturnCode code = checker.canTakeOperation(loc1, maxHeapSizePerRequest);
1338 assertEquals(RowChecker.ReturnCode.INCLUDE, code);
1339 code = checker.canTakeOperation(loc2, maxHeapSizePerRequest);
1340 assertEquals(RowChecker.ReturnCode.INCLUDE, code);
1341 }
1342 }
1343
1344 @Test
1345 public void testSubmittedSizeChecker() {
1346 final long maxHeapSizeSubmit = 2 * 1024 * 1024;
1347 SubmittedSizeChecker checker = new SubmittedSizeChecker(maxHeapSizeSubmit);
1348
1349 for (int i = 0; i != 10; ++i) {
1350 ReturnCode include = checker.canTakeOperation(loc1, 100000);
1351 assertEquals(ReturnCode.INCLUDE, include);
1352 }
1353
1354 for (int i = 0; i != 10; ++i) {
1355 checker.notifyFinal(ReturnCode.INCLUDE, loc1, maxHeapSizeSubmit);
1356 }
1357
1358 for (int i = 0; i != 10; ++i) {
1359 ReturnCode include = checker.canTakeOperation(loc1, 100000);
1360 assertEquals(ReturnCode.END, include);
1361 }
1362 for (int i = 0; i != 10; ++i) {
1363 ReturnCode include = checker.canTakeOperation(loc2, 100000);
1364 assertEquals(ReturnCode.END, include);
1365 }
1366 checker.reset();
1367 for (int i = 0; i != 10; ++i) {
1368 ReturnCode include = checker.canTakeOperation(loc1, 100000);
1369 assertEquals(ReturnCode.INCLUDE, include);
1370 }
1371 }
1372 @Test
1373 public void testTaskCountChecker() throws InterruptedIOException {
1374 long rowSize = 12345;
1375 int maxTotalConcurrentTasks = 100;
1376 int maxConcurrentTasksPerServer = 2;
1377 int maxConcurrentTasksPerRegion = 1;
1378 AtomicLong tasksInProgress = new AtomicLong(0);
1379 Map<ServerName, AtomicInteger> taskCounterPerServer = new HashMap<>();
1380 Map<byte[], AtomicInteger> taskCounterPerRegion = new HashMap<>();
1381 TaskCountChecker checker = new TaskCountChecker(
1382 maxTotalConcurrentTasks,
1383 maxConcurrentTasksPerServer,
1384 maxConcurrentTasksPerRegion,
1385 tasksInProgress, taskCounterPerServer, taskCounterPerRegion);
1386
1387
1388 for (int i = 0; i != 10; ++i) {
1389 ReturnCode code = checker.canTakeOperation(loc1, rowSize);
1390 assertEquals(RowChecker.ReturnCode.INCLUDE, code);
1391 }
1392
1393 ReturnCode code = checker.canTakeOperation(loc1, rowSize);
1394 assertEquals(RowChecker.ReturnCode.INCLUDE, code);
1395 checker.notifyFinal(code, loc1, rowSize);
1396
1397
1398 taskCounterPerRegion.put(loc1.getRegionInfo().getRegionName(), new AtomicInteger(100));
1399 taskCounterPerServer.put(loc1.getServerName(), new AtomicInteger(100));
1400
1401
1402 for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
1403 ReturnCode includeCode = checker.canTakeOperation(loc1, rowSize);
1404 assertEquals(RowChecker.ReturnCode.INCLUDE, includeCode);
1405 checker.notifyFinal(includeCode, loc1, rowSize);
1406 }
1407
1408
1409 taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(100));
1410 taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(100));
1411
1412
1413 for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
1414 ReturnCode excludeCode = checker.canTakeOperation(loc3, rowSize);
1415 assertNotEquals(RowChecker.ReturnCode.INCLUDE, excludeCode);
1416 checker.notifyFinal(excludeCode, loc3, rowSize);
1417 }
1418
1419
1420 taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(0));
1421 taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(0));
1422
1423
1424 ReturnCode code3 = checker.canTakeOperation(loc3, rowSize);
1425 assertEquals(RowChecker.ReturnCode.INCLUDE, code3);
1426 checker.notifyFinal(code3, loc3, rowSize);
1427
1428
1429 for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
1430 ReturnCode includeCode = checker.canTakeOperation(loc3, rowSize);
1431 assertEquals(RowChecker.ReturnCode.INCLUDE, includeCode);
1432 checker.notifyFinal(includeCode, loc3, rowSize);
1433 }
1434
1435 checker.reset();
1436
1437
1438
1439 for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
1440 ReturnCode includeCode = checker.canTakeOperation(loc1, rowSize);
1441 assertNotEquals(RowChecker.ReturnCode.INCLUDE, includeCode);
1442 checker.notifyFinal(includeCode, loc1, rowSize);
1443 }
1444 }
1445
1446 @Test
1447 public void testBatch() throws IOException, InterruptedException {
1448 ClusterConnection conn = new MyConnectionImpl(conf);
1449 HTable ht = new HTable(conn, new BufferedMutatorParams(DUMMY_TABLE));
1450 ht.multiAp = new MyAsyncProcess(conn, conf, false);
1451
1452 List<Put> puts = new ArrayList<Put>();
1453 puts.add(createPut(1, true));
1454 puts.add(createPut(1, true));
1455 puts.add(createPut(1, true));
1456 puts.add(createPut(1, true));
1457 puts.add(createPut(1, false));
1458 puts.add(createPut(1, true));
1459 puts.add(createPut(1, false));
1460
1461 Object[] res = new Object[puts.size()];
1462 try {
1463 ht.batch(puts, res);
1464 Assert.fail();
1465 } catch (RetriesExhaustedException expected) {
1466 }
1467
1468 Assert.assertEquals(success, res[0]);
1469 Assert.assertEquals(success, res[1]);
1470 Assert.assertEquals(success, res[2]);
1471 Assert.assertEquals(success, res[3]);
1472 Assert.assertEquals(failure, res[4]);
1473 Assert.assertEquals(success, res[5]);
1474 Assert.assertEquals(failure, res[6]);
1475 }
1476 @Test
1477 public void testErrorsServers() throws IOException {
1478 Configuration configuration = new Configuration(conf);
1479 ClusterConnection conn = new MyConnectionImpl(configuration);
1480 BufferedMutatorImpl mutator =
1481 new BufferedMutatorImpl(conn, null, null, new BufferedMutatorParams(DUMMY_TABLE));
1482 configuration.setBoolean(ConnectionManager.RETRIES_BY_SERVER_KEY, true);
1483
1484 MyAsyncProcess ap = new MyAsyncProcess(conn, configuration, true);
1485 mutator.ap = ap;
1486
1487 Assert.assertNotNull(mutator.ap.createServerErrorTracker());
1488 Assert.assertTrue(mutator.ap.serverTrackerTimeout > 200);
1489 mutator.ap.serverTrackerTimeout = 1;
1490
1491 Put p = createPut(1, false);
1492 mutator.mutate(p);
1493
1494 try {
1495 mutator.flush();
1496 Assert.fail();
1497 } catch (RetriesExhaustedWithDetailsException expected) {
1498 }
1499
1500 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
1501 }
1502
1503 @Test
1504 public void testReadAndWriteTimeout() throws IOException {
1505 final long readTimeout = 10 * 1000;
1506 final long writeTimeout = 20 * 1000;
1507 Configuration copyConf = new Configuration(conf);
1508 copyConf.setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, readTimeout);
1509 copyConf.setLong(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, writeTimeout);
1510 ClusterConnection conn = createHConnection();
1511 Mockito.when(conn.getConfiguration()).thenReturn(copyConf);
1512 BufferedMutatorParams bufferParam = new BufferedMutatorParams(DUMMY_TABLE);
1513 try (HTable ht = new HTable(conn, bufferParam)) {
1514 MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, true);
1515 ht.multiAp = ap;
1516 List<Get> gets = new LinkedList<>();
1517 gets.add(new Get(DUMMY_BYTES_1));
1518 gets.add(new Get(DUMMY_BYTES_2));
1519 try {
1520 ht.get(gets);
1521 } catch (ClassCastException e) {
1522
1523 }
1524 assertEquals(readTimeout, ap.previousTimeout);
1525 ap.previousTimeout = -1;
1526
1527 try {
1528 ht.existsAll(gets);
1529 } catch (ClassCastException e) {
1530
1531 }
1532 assertEquals(readTimeout, ap.previousTimeout);
1533 ap.previousTimeout = -1;
1534
1535 List<Delete> deletes = new LinkedList<>();
1536 deletes.add(new Delete(DUMMY_BYTES_1));
1537 deletes.add(new Delete(DUMMY_BYTES_2));
1538 ht.delete(deletes);
1539 assertEquals(writeTimeout, ap.previousTimeout);
1540 }
1541 }
1542
1543 @Test
1544 public void testGlobalErrors() throws IOException {
1545 ClusterConnection conn = new MyConnectionImpl(conf);
1546 BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(DUMMY_TABLE);
1547 AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, conf, new IOException("test"));
1548 mutator.ap = ap;
1549
1550 Assert.assertNotNull(mutator.ap.createServerErrorTracker());
1551
1552 Put p = createPut(1, true);
1553 mutator.mutate(p);
1554
1555 try {
1556 mutator.flush();
1557 Assert.fail();
1558 } catch (RetriesExhaustedWithDetailsException expected) {
1559 }
1560
1561 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
1562 }
1563
1564
1565 @Test
1566 public void testCallQueueTooLarge() throws IOException {
1567 ClusterConnection conn = new MyConnectionImpl(conf);
1568 BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(DUMMY_TABLE);
1569 AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, conf, new CallQueueTooBigException());
1570 mutator.ap = ap;
1571
1572 Assert.assertNotNull(mutator.ap.createServerErrorTracker());
1573
1574 Put p = createPut(1, true);
1575 mutator.mutate(p);
1576
1577 try {
1578 mutator.flush();
1579 Assert.fail();
1580 } catch (RetriesExhaustedWithDetailsException expected) {
1581 }
1582
1583 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
1584 }
1585
1586
1587
1588
1589 @Test
1590 public void testThreadCreation() throws Exception {
1591 final int NB_REGS = 100;
1592 List<HRegionLocation> hrls = new ArrayList<HRegionLocation>(NB_REGS);
1593 List<Get> gets = new ArrayList<Get>(NB_REGS);
1594 for (int i = 0; i < NB_REGS; i++) {
1595 HRegionInfo hri = new HRegionInfo(
1596 DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i);
1597 HRegionLocation hrl = new HRegionLocation(hri, i % 2 == 0 ? sn : sn2);
1598 hrls.add(hrl);
1599
1600 Get get = new Get(Bytes.toBytes(i * 10L));
1601 gets.add(get);
1602 }
1603
1604 MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
1605 HTable ht = new HTable(con, new BufferedMutatorParams(DUMMY_TABLE));
1606 MyAsyncProcess ap = new MyAsyncProcess(con, conf, con.nbThreads);
1607 ht.multiAp = ap;
1608
1609 ht.batch(gets, new Object[gets.size()]);
1610
1611 Assert.assertEquals(NB_REGS, ap.nbActions.get());
1612 Assert.assertEquals("1 multi response per server", 2, ap.nbMultiResponse.get());
1613 Assert.assertEquals("1 thread per server", 2, con.nbThreads.get());
1614
1615 int nbReg = 0;
1616 for (int i =0; i<NB_REGS; i++){
1617 if (con.usedRegions[i]) nbReg++;
1618 }
1619 Assert.assertEquals("nbReg=" + nbReg, NB_REGS, nbReg);
1620 }
1621
1622 @Test
1623 public void testReplicaReplicaSuccess() throws Exception {
1624
1625
1626 MyAsyncProcessWithReplicas ap = createReplicaAp(10, 1000, 0);
1627 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3);
1628 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[3]);
1629 verifyReplicaResult(ars, RR.TRUE, RR.TRUE, RR.FALSE);
1630 Assert.assertEquals(2, ap.getReplicaCallCount());
1631 }
1632
1633 @Test
1634 public void testReplicaPrimarySuccessWoReplicaCalls() throws Exception {
1635
1636 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 10, 0);
1637 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3);
1638 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[3]);
1639 verifyReplicaResult(ars, RR.FALSE, RR.FALSE, RR.FALSE);
1640 Assert.assertEquals(0, ap.getReplicaCallCount());
1641 }
1642
1643 @Test
1644 public void testReplicaParallelCallsSucceed() throws Exception {
1645
1646 MyAsyncProcessWithReplicas ap = createReplicaAp(0, 0, 0);
1647 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
1648 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
1649 verifyReplicaResult(ars, RR.DONT_CARE, RR.DONT_CARE);
1650 long replicaCalls = ap.getReplicaCallCount();
1651 Assert.assertTrue(replicaCalls >= 0);
1652 Assert.assertTrue(replicaCalls <= 2);
1653 }
1654
1655 @Test
1656 public void testReplicaPartialReplicaCall() throws Exception {
1657
1658
1659
1660 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0);
1661 ap.setPrimaryCallDelay(sn2, 2000);
1662 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
1663 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
1664 verifyReplicaResult(ars, RR.FALSE, RR.TRUE);
1665 Assert.assertEquals(1, ap.getReplicaCallCount());
1666 }
1667
1668 @Test
1669 public void testReplicaMainFailsBeforeReplicaCalls() throws Exception {
1670
1671
1672
1673 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0, 1);
1674 ap.addFailures(hri1, hri2);
1675 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
1676 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
1677 verifyReplicaResult(ars, RR.FAILED, RR.FAILED);
1678 Assert.assertEquals(0, ap.getReplicaCallCount());
1679 }
1680
1681 @Test
1682 public void testReplicaReplicaSuccessWithParallelFailures() throws Exception {
1683
1684
1685 MyAsyncProcessWithReplicas ap = createReplicaAp(0, 1000, 1000, 1);
1686 ap.addFailures(hri1, hri1r2, hri2);
1687 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
1688 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
1689 verifyReplicaResult(ars, RR.TRUE, RR.TRUE);
1690 Assert.assertEquals(2, ap.getReplicaCallCount());
1691 }
1692
1693 @Test
1694 public void testReplicaAllCallsFailForOneRegion() throws Exception {
1695
1696
1697 MyAsyncProcessWithReplicas ap = createReplicaAp(500, 1000, 0, 1);
1698 ap.addFailures(hri1, hri1r1, hri1r2, hri2r1);
1699 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2);
1700 AsyncRequestFuture ars = ap.submitAll(DUMMY_TABLE, rows, null, new Object[2]);
1701 verifyReplicaResult(ars, RR.FAILED, RR.FALSE);
1702
1703 Assert.assertEquals(3, ars.getErrors().getNumExceptions());
1704 for (int i = 0; i < ars.getErrors().getNumExceptions(); ++i) {
1705 Assert.assertArrayEquals(DUMMY_BYTES_1, ars.getErrors().getRow(i).getRow());
1706 }
1707 }
1708
1709 private MyAsyncProcessWithReplicas createReplicaAp(
1710 int replicaAfterMs, int primaryMs, int replicaMs) throws Exception {
1711 return createReplicaAp(replicaAfterMs, primaryMs, replicaMs, -1);
1712 }
1713
1714 private MyAsyncProcessWithReplicas createReplicaAp(
1715 int replicaAfterMs, int primaryMs, int replicaMs, int retries) throws Exception {
1716
1717
1718 Configuration conf = new Configuration();
1719 ClusterConnection conn = createHConnectionWithReplicas();
1720 conf.setInt(AsyncProcess.PRIMARY_CALL_TIMEOUT_KEY, replicaAfterMs * 1000);
1721 if (retries > 0) {
1722 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
1723 }
1724 MyAsyncProcessWithReplicas ap = new MyAsyncProcessWithReplicas(conn, conf);
1725 ap.setCallDelays(primaryMs, replicaMs);
1726 return ap;
1727 }
1728
1729 private static BufferedMutatorParams createBufferedMutatorParams(MyAsyncProcess ap,
1730 TableName name) {
1731 return new BufferedMutatorParams(name)
1732 .pool(ap.pool);
1733 }
1734
1735 private static List<Get> makeTimelineGets(byte[]... rows) {
1736 List<Get> result = new ArrayList<Get>();
1737 for (byte[] row : rows) {
1738 Get get = new Get(row);
1739 get.setConsistency(Consistency.TIMELINE);
1740 result.add(get);
1741 }
1742 return result;
1743 }
1744
1745 private void verifyResult(AsyncRequestFuture ars, boolean... expected) throws Exception {
1746 Object[] actual = ars.getResults();
1747 Assert.assertEquals(expected.length, actual.length);
1748 for (int i = 0; i < expected.length; ++i) {
1749 Assert.assertEquals(expected[i], !(actual[i] instanceof Throwable));
1750 }
1751 }
1752
1753
1754 private enum RR {
1755 TRUE,
1756 FALSE,
1757 DONT_CARE,
1758 FAILED
1759 }
1760
1761 private void verifyReplicaResult(AsyncRequestFuture ars, RR... expecteds) throws Exception {
1762 Object[] actuals = ars.getResults();
1763 Assert.assertEquals(expecteds.length, actuals.length);
1764 for (int i = 0; i < expecteds.length; ++i) {
1765 Object actual = actuals[i];
1766 RR expected = expecteds[i];
1767 Assert.assertEquals(actual.toString(), expected == RR.FAILED, actual instanceof Throwable);
1768 if (expected != RR.FAILED && expected != RR.DONT_CARE) {
1769 Assert.assertEquals(expected == RR.TRUE, ((Result)actual).isStale());
1770 }
1771 }
1772 }
1773
1774
1775
1776
1777
1778
1779 private Put createPut(int regCnt, boolean success) {
1780 Put p;
1781 if (!success) {
1782 p = new Put(FAILS);
1783 } else switch (regCnt){
1784 case 1 :
1785 p = new Put(DUMMY_BYTES_1);
1786 break;
1787 case 2:
1788 p = new Put(DUMMY_BYTES_2);
1789 break;
1790 case 3:
1791 p = new Put(DUMMY_BYTES_3);
1792 break;
1793 default:
1794 throw new IllegalArgumentException("unknown " + regCnt);
1795 }
1796
1797 p.add(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
1798
1799 return p;
1800 }
1801
1802 static class MyThreadPoolExecutor extends ThreadPoolExecutor {
1803 public MyThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime,
1804 TimeUnit timeunit, BlockingQueue<Runnable> blockingqueue) {
1805 super(coreThreads, maxThreads, keepAliveTime, timeunit, blockingqueue);
1806 }
1807
1808 @Override
1809 public Future submit(Runnable runnable) {
1810 throw new OutOfMemoryError("OutOfMemory error thrown by means");
1811 }
1812 }
1813
1814 static class AsyncProcessForThrowableCheck extends AsyncProcess {
1815 private static int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
1816
1817 public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf,
1818 ExecutorService pool) {
1819 super(hc, conf, pool, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(
1820 conf), rpcTimeout);
1821 }
1822 }
1823
1824 @Test
1825 public void testUncheckedException() throws Exception {
1826
1827 ClusterConnection hc = createHConnection();
1828 MyThreadPoolExecutor myPool =
1829 new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
1830 new LinkedBlockingQueue<Runnable>(200));
1831 AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, conf, myPool);
1832
1833 List<Put> puts = new ArrayList<Put>();
1834 puts.add(createPut(1, true));
1835
1836 ap.submit(DUMMY_TABLE, puts, false, null, false);
1837 Assert.assertTrue(puts.isEmpty());
1838 }
1839
1840 @Test
1841 public void testWaitForMaximumCurrentTasks() throws Exception {
1842 final AtomicLong tasks = new AtomicLong(0);
1843 final AtomicInteger max = new AtomicInteger(0);
1844 final CyclicBarrier barrier = new CyclicBarrier(2);
1845 final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf);
1846 Runnable runnable = new Runnable() {
1847 @Override
1848 public void run() {
1849 try {
1850 barrier.await();
1851 ap.waitForMaximumCurrentTasks(max.get(), tasks, 1, null);
1852 } catch (InterruptedIOException e) {
1853 Assert.fail(e.getMessage());
1854 } catch (InterruptedException e) {
1855
1856 e.printStackTrace();
1857 } catch (BrokenBarrierException e) {
1858
1859 e.printStackTrace();
1860 }
1861 }
1862 };
1863
1864 Thread t = new Thread(runnable);
1865 t.start();
1866 barrier.await();
1867 t.join();
1868
1869 barrier.reset();
1870 tasks.set(1000000);
1871 t = new Thread(runnable);
1872 t.start();
1873 barrier.await();
1874 while (tasks.get() > 0) {
1875 assertTrue(t.isAlive());
1876 tasks.set(tasks.get() - 1);
1877 }
1878 t.join();
1879 }
1880
1881
1882
1883
1884
1885
1886 @Test
1887 public void testRetryPauseWithCallQueueTooBigException() throws Exception {
1888 Configuration myConf = new Configuration(conf);
1889 final long specialPause = 500L;
1890 final int tries = 2;
1891 myConf.setLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, specialPause);
1892 myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, tries);
1893 ClusterConnection conn = new MyConnectionImpl(myConf);
1894 BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(DUMMY_TABLE);
1895 AsyncProcessWithFailure ap =
1896 new AsyncProcessWithFailure(conn, myConf, new CallQueueTooBigException());
1897 mutator.ap = ap;
1898
1899 Assert.assertNotNull(mutator.ap.createServerErrorTracker());
1900
1901 Put p = createPut(1, true);
1902 mutator.mutate(p);
1903
1904 long startTime = System.currentTimeMillis();
1905 try {
1906 mutator.flush();
1907 Assert.fail();
1908 } catch (RetriesExhaustedWithDetailsException expected) {
1909 }
1910 long actualSleep = System.currentTimeMillis() - startTime;
1911 long expectedSleep = 0L;
1912 for (int i = 0; i < tries - 1; i++) {
1913 expectedSleep += ConnectionUtils.getPauseTime(specialPause, i);
1914
1915 actualSleep += (long) (specialPause * 0.01f);
1916 }
1917 LOG.debug("Expected to sleep " + expectedSleep + "ms, actually slept " + actualSleep + "ms");
1918 Assert.assertTrue("Expected to sleep " + expectedSleep + " but actually " + actualSleep + "ms",
1919 actualSleep >= expectedSleep);
1920
1921
1922 final long normalPause =
1923 myConf.getLong(HConstants.HBASE_CLIENT_PAUSE, HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
1924 ap = new AsyncProcessWithFailure(conn, myConf, new IOException());
1925 mutator.ap = ap;
1926 Assert.assertNotNull(mutator.ap.createServerErrorTracker());
1927 mutator.mutate(p);
1928 startTime = System.currentTimeMillis();
1929 try {
1930 mutator.flush();
1931 Assert.fail();
1932 } catch (RetriesExhaustedWithDetailsException expected) {
1933 }
1934 actualSleep = System.currentTimeMillis() - startTime;
1935 expectedSleep = 0L;
1936 for (int i = 0; i < tries - 1; i++) {
1937 expectedSleep += ConnectionUtils.getPauseTime(normalPause, i);
1938 }
1939
1940 expectedSleep += normalPause;
1941 LOG.debug("Expected to sleep " + expectedSleep + "ms, actually slept " + actualSleep + "ms");
1942 Assert.assertTrue("Slept for too long: " + actualSleep + "ms", actualSleep <= expectedSleep);
1943 }
1944
1945 @Test
1946 public void testQueueRowAccess() throws Exception {
1947 ClusterConnection conn = createHConnection();
1948 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null,
1949 new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(100000));
1950 Put p0 = new Put(DUMMY_BYTES_1).addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
1951 Put p1 = new Put(DUMMY_BYTES_2).addColumn(DUMMY_BYTES_2, DUMMY_BYTES_2, DUMMY_BYTES_2);
1952 mutator.mutate(p0);
1953 BufferedMutatorImpl.QueueRowAccess ra0 = mutator.createQueueRowAccess();
1954
1955 assertEquals(0, mutator.undealtMutationCount.get());
1956 mutator.mutate(p1);
1957 assertEquals(1, mutator.undealtMutationCount.get());
1958 BufferedMutatorImpl.QueueRowAccess ra1 = mutator.createQueueRowAccess();
1959
1960 assertEquals(0, mutator.undealtMutationCount.get());
1961 assertEquals(1, ra0.size());
1962 assertEquals(1, ra1.size());
1963 Iterator<Row> iter0 = ra0.iterator();
1964 Iterator<Row> iter1 = ra1.iterator();
1965 assertTrue(iter0.hasNext());
1966 assertTrue(iter1.hasNext());
1967
1968 assertTrue(iter0.next() == p0);
1969 assertEquals(1, mutator.writeAsyncBuffer.size());
1970 assertEquals(p1.heapSize(), mutator.currentWriteBufferSize.get());
1971 assertTrue(iter1.next() == p1);
1972 assertEquals(0, mutator.writeAsyncBuffer.size());
1973 assertEquals(0, mutator.currentWriteBufferSize.get());
1974 assertFalse(iter0.hasNext());
1975 assertFalse(iter1.hasNext());
1976
1977 iter0.remove();
1978 ra0.close();
1979 assertEquals(0, mutator.undealtMutationCount.get());
1980 assertEquals(0, mutator.writeAsyncBuffer.size());
1981 assertEquals(0, mutator.currentWriteBufferSize.get());
1982
1983 ra1.close();
1984 assertEquals(1, mutator.undealtMutationCount.get());
1985 assertEquals(1, mutator.writeAsyncBuffer.size());
1986 assertEquals(p1.heapSize(), mutator.currentWriteBufferSize.get());
1987 }
1988 }