001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.client; 019 020import static org.junit.Assert.assertEquals; 021import static org.junit.Assert.assertFalse; 022import static org.junit.Assert.assertTrue; 023 024import java.io.IOException; 025import java.io.InterruptedIOException; 026import java.util.ArrayList; 027import java.util.Arrays; 028import java.util.Collections; 029import java.util.HashMap; 030import java.util.Iterator; 031import java.util.LinkedList; 032import java.util.List; 033import java.util.Map; 034import java.util.Random; 035import java.util.Set; 036import java.util.TreeSet; 037import java.util.concurrent.BlockingQueue; 038import java.util.concurrent.CompletableFuture; 039import java.util.concurrent.ExecutorService; 040import java.util.concurrent.Executors; 041import java.util.concurrent.Future; 042import java.util.concurrent.LinkedBlockingQueue; 043import java.util.concurrent.SynchronousQueue; 044import java.util.concurrent.ThreadFactory; 045import java.util.concurrent.ThreadPoolExecutor; 046import java.util.concurrent.TimeUnit; 047import java.util.concurrent.atomic.AtomicBoolean; 048import java.util.concurrent.atomic.AtomicInteger; 049import java.util.concurrent.atomic.AtomicLong; 050import org.apache.hadoop.conf.Configuration; 051import org.apache.hadoop.hbase.CallQueueTooBigException; 052import org.apache.hadoop.hbase.Cell; 053import org.apache.hadoop.hbase.HBaseClassTestRule; 054import org.apache.hadoop.hbase.HConstants; 055import org.apache.hadoop.hbase.HRegionInfo; 056import org.apache.hadoop.hbase.HRegionLocation; 057import org.apache.hadoop.hbase.RegionLocations; 058import org.apache.hadoop.hbase.ServerName; 059import org.apache.hadoop.hbase.TableName; 060import org.apache.hadoop.hbase.client.AsyncProcessTask.ListRowAccess; 061import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows; 062import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; 063import org.apache.hadoop.hbase.client.backoff.ServerStatistics; 064import org.apache.hadoop.hbase.client.coprocessor.Batch; 065import org.apache.hadoop.hbase.ipc.RpcControllerFactory; 066import org.apache.hadoop.hbase.testclassification.ClientTests; 067import org.apache.hadoop.hbase.testclassification.MediumTests; 068import org.apache.hadoop.hbase.util.Bytes; 069import org.apache.hadoop.hbase.util.Threads; 070import org.junit.Assert; 071import org.junit.BeforeClass; 072import org.junit.ClassRule; 073import org.junit.Test; 074import org.junit.experimental.categories.Category; 075import org.mockito.Mockito; 076import org.slf4j.Logger; 077import org.slf4j.LoggerFactory; 078 079@Category({ClientTests.class, MediumTests.class}) 080public class TestAsyncProcess { 081 082 @ClassRule 083 public static final HBaseClassTestRule CLASS_RULE = 084 HBaseClassTestRule.forClass(TestAsyncProcess.class); 085 086 private static final Logger LOG = LoggerFactory.getLogger(TestAsyncProcess.class); 087 private static final TableName DUMMY_TABLE = 088 TableName.valueOf("DUMMY_TABLE"); 089 private static final byte[] DUMMY_BYTES_1 = Bytes.toBytes("DUMMY_BYTES_1"); 090 private static final byte[] DUMMY_BYTES_2 = Bytes.toBytes("DUMMY_BYTES_2"); 091 private static final byte[] DUMMY_BYTES_3 = Bytes.toBytes("DUMMY_BYTES_3"); 092 private static final byte[] FAILS = Bytes.toBytes("FAILS"); 093 private static final Configuration CONF = new Configuration(); 094 private static final ConnectionConfiguration CONNECTION_CONFIG = 095 new ConnectionConfiguration(CONF); 096 private static final ServerName sn = ServerName.valueOf("s1,1,1"); 097 private static final ServerName sn2 = ServerName.valueOf("s2,2,2"); 098 private static final ServerName sn3 = ServerName.valueOf("s3,3,3"); 099 private static final HRegionInfo hri1 = 100 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1); 101 private static final HRegionInfo hri2 = 102 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_2, HConstants.EMPTY_END_ROW, false, 2); 103 private static final HRegionInfo hri3 = 104 new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_3, HConstants.EMPTY_END_ROW, false, 3); 105 private static final HRegionLocation loc1 = new HRegionLocation(hri1, sn); 106 private static final HRegionLocation loc2 = new HRegionLocation(hri2, sn); 107 private static final HRegionLocation loc3 = new HRegionLocation(hri3, sn2); 108 109 // Replica stuff 110 private static final RegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1); 111 private static final RegionInfo hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2); 112 private static final RegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1); 113 private static final RegionLocations hrls1 = new RegionLocations(new HRegionLocation(hri1, sn), 114 new HRegionLocation(hri1r1, sn2), new HRegionLocation(hri1r2, sn3)); 115 private static final RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2), 116 new HRegionLocation(hri2r1, sn3)); 117 private static final RegionLocations hrls3 = 118 new RegionLocations(new HRegionLocation(hri3, sn3), null); 119 120 private static final String success = "success"; 121 private static Exception failure = new Exception("failure"); 122 123 private static final int NB_RETRIES = 3; 124 125 private static final int RPC_TIMEOUT = CONF.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 126 HConstants.DEFAULT_HBASE_RPC_TIMEOUT); 127 private static final int OPERATION_TIMEOUT = CONF.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 128 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); 129 @BeforeClass 130 public static void beforeClass(){ 131 CONF.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, NB_RETRIES); 132 } 133 134 static class CountingThreadFactory implements ThreadFactory { 135 final AtomicInteger nbThreads; 136 ThreadFactory realFactory = Threads.newDaemonThreadFactory("test-TestAsyncProcess"); 137 @Override 138 public Thread newThread(Runnable r) { 139 nbThreads.incrementAndGet(); 140 return realFactory.newThread(r); 141 } 142 143 CountingThreadFactory(AtomicInteger nbThreads){ 144 this.nbThreads = nbThreads; 145 } 146 } 147 148 static class MyAsyncProcess extends AsyncProcess { 149 final AtomicInteger nbMultiResponse = new AtomicInteger(); 150 final AtomicInteger nbActions = new AtomicInteger(); 151 public List<AsyncRequestFuture> allReqs = new ArrayList<>(); 152 public AtomicInteger callsCt = new AtomicInteger(); 153 154 private long previousTimeout = -1; 155 final ExecutorService service; 156 @Override 157 protected <Res> AsyncRequestFutureImpl<Res> createAsyncRequestFuture( 158 AsyncProcessTask task, List<Action> actions, long nonceGroup) { 159 // Test HTable has tableName of null, so pass DUMMY_TABLE 160 AsyncProcessTask wrap = new AsyncProcessTask(task){ 161 @Override 162 public TableName getTableName() { 163 return DUMMY_TABLE; 164 } 165 }; 166 AsyncRequestFutureImpl<Res> r = new MyAsyncRequestFutureImpl<>( 167 wrap, actions, nonceGroup, this); 168 allReqs.add(r); 169 return r; 170 } 171 172 public MyAsyncProcess(ClusterConnection hc, Configuration conf) { 173 super(hc, conf, 174 new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf)); 175 service = Executors.newFixedThreadPool(5); 176 } 177 178 public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) { 179 super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf)); 180 service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, 181 new SynchronousQueue<>(), new CountingThreadFactory(nbThreads)); 182 } 183 184 public <CResult> AsyncRequestFuture submit(ExecutorService pool, TableName tableName, 185 List<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback, 186 boolean needResults) throws InterruptedIOException { 187 AsyncProcessTask task = AsyncProcessTask.newBuilder(callback) 188 .setPool(pool == null ? service : pool) 189 .setTableName(tableName) 190 .setRowAccess(rows) 191 .setSubmittedRows(atLeastOne ? SubmittedRows.AT_LEAST_ONE : SubmittedRows.NORMAL) 192 .setNeedResults(needResults) 193 .setRpcTimeout(RPC_TIMEOUT) 194 .setOperationTimeout(OPERATION_TIMEOUT) 195 .build(); 196 return submit(task); 197 } 198 199 public <CResult> AsyncRequestFuture submit(TableName tableName, 200 final List<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback, 201 boolean needResults) throws InterruptedIOException { 202 return submit(null, tableName, rows, atLeastOne, callback, needResults); 203 } 204 205 @Override 206 public <Res> AsyncRequestFuture submit(AsyncProcessTask<Res> task) 207 throws InterruptedIOException { 208 previousTimeout = task.getRpcTimeout(); 209 // We use results in tests to check things, so override to always save them. 210 AsyncProcessTask<Res> wrap = new AsyncProcessTask<Res>(task) { 211 @Override 212 public boolean getNeedResults() { 213 return true; 214 } 215 }; 216 return super.submit(wrap); 217 } 218 219 @Override 220 protected RpcRetryingCaller<AbstractResponse> createCaller( 221 CancellableRegionServerCallable callable, int rpcTimeout) { 222 callsCt.incrementAndGet(); 223 MultiServerCallable callable1 = (MultiServerCallable) callable; 224 final MultiResponse mr = createMultiResponse( 225 callable1.getMulti(), nbMultiResponse, nbActions, 226 new ResponseGenerator() { 227 @Override 228 public void addResponse(MultiResponse mr, byte[] regionName, Action a) { 229 if (Arrays.equals(FAILS, a.getAction().getRow())) { 230 mr.add(regionName, a.getOriginalIndex(), failure); 231 } else { 232 mr.add(regionName, a.getOriginalIndex(), success); 233 } 234 } 235 }); 236 237 return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 10, 9) { 238 @Override 239 public AbstractResponse callWithoutRetries(RetryingCallable<AbstractResponse> callable, 240 int callTimeout) 241 throws IOException, RuntimeException { 242 try { 243 // sleep one second in order for threadpool to start another thread instead of reusing 244 // existing one. 245 Thread.sleep(1000); 246 } catch (InterruptedException e) { 247 // ignore error 248 } 249 return mr; 250 } 251 }; 252 } 253 254 255 } 256 257 static class MyAsyncRequestFutureImpl<Res> extends AsyncRequestFutureImpl<Res> { 258 private final Map<ServerName, List<Long>> heapSizesByServer = new HashMap<>(); 259 public MyAsyncRequestFutureImpl(AsyncProcessTask task, List<Action> actions, 260 long nonceGroup, AsyncProcess asyncProcess) { 261 super(task, actions, nonceGroup, asyncProcess); 262 } 263 264 @Override 265 protected void updateStats(ServerName server, Map<byte[], MultiResponse.RegionResult> results) { 266 // Do nothing for avoiding the NPE if we test the ClientBackofPolicy. 267 } 268 269 Map<ServerName, List<Long>> getRequestHeapSize() { 270 return heapSizesByServer; 271 } 272 273 @Override 274 SingleServerRequestRunnable createSingleServerRequest( 275 MultiAction multiAction, int numAttempt, ServerName server, 276 Set<CancellableRegionServerCallable> callsInProgress) { 277 SingleServerRequestRunnable rq = new SingleServerRequestRunnable( 278 multiAction, numAttempt, server, callsInProgress); 279 List<Long> heapCount = heapSizesByServer.get(server); 280 if (heapCount == null) { 281 heapCount = new ArrayList<>(); 282 heapSizesByServer.put(server, heapCount); 283 } 284 heapCount.add(heapSizeOf(multiAction)); 285 return rq; 286 } 287 288 private long heapSizeOf(MultiAction multiAction) { 289 return multiAction.actions.values().stream() 290 .flatMap(v -> v.stream()) 291 .map(action -> action.getAction()) 292 .filter(row -> row instanceof Mutation) 293 .mapToLong(row -> ((Mutation) row).heapSize()) 294 .sum(); 295 } 296 } 297 298 static class CallerWithFailure extends RpcRetryingCallerImpl<AbstractResponse>{ 299 300 private final IOException e; 301 302 public CallerWithFailure(IOException e) { 303 super(100, 500, 100, 9); 304 this.e = e; 305 } 306 307 @Override 308 public AbstractResponse callWithoutRetries(RetryingCallable<AbstractResponse> callable, 309 int callTimeout) 310 throws IOException, RuntimeException { 311 throw e; 312 } 313 } 314 315 316 static class AsyncProcessWithFailure extends MyAsyncProcess { 317 318 private final IOException ioe; 319 320 public AsyncProcessWithFailure(ClusterConnection hc, Configuration conf, IOException ioe) { 321 super(hc, conf); 322 this.ioe = ioe; 323 serverTrackerTimeout = 1L; 324 } 325 326 @Override 327 protected RpcRetryingCaller<AbstractResponse> createCaller( 328 CancellableRegionServerCallable callable, int rpcTimeout) { 329 callsCt.incrementAndGet(); 330 return new CallerWithFailure(ioe); 331 } 332 } 333 334 /** 335 * Make the backoff time always different on each call. 336 */ 337 static class MyClientBackoffPolicy implements ClientBackoffPolicy { 338 private final Map<ServerName, AtomicInteger> count = new HashMap<>(); 339 @Override 340 public long getBackoffTime(ServerName serverName, byte[] region, ServerStatistics stats) { 341 AtomicInteger inc = count.get(serverName); 342 if (inc == null) { 343 inc = new AtomicInteger(0); 344 count.put(serverName, inc); 345 } 346 return inc.getAndIncrement(); 347 } 348 } 349 350 static class MyAsyncProcessWithReplicas extends MyAsyncProcess { 351 private Set<byte[]> failures = new TreeSet<>(new Bytes.ByteArrayComparator()); 352 private long primarySleepMs = 0, replicaSleepMs = 0; 353 private Map<ServerName, Long> customPrimarySleepMs = new HashMap<>(); 354 private final AtomicLong replicaCalls = new AtomicLong(0); 355 356 public void addFailures(RegionInfo... hris) { 357 for (RegionInfo hri : hris) { 358 failures.add(hri.getRegionName()); 359 } 360 } 361 362 public long getReplicaCallCount() { 363 return replicaCalls.get(); 364 } 365 366 public void setPrimaryCallDelay(ServerName server, long primaryMs) { 367 customPrimarySleepMs.put(server, primaryMs); 368 } 369 370 public MyAsyncProcessWithReplicas(ClusterConnection hc, Configuration conf) { 371 super(hc, conf); 372 } 373 374 public void setCallDelays(long primaryMs, long replicaMs) { 375 this.primarySleepMs = primaryMs; 376 this.replicaSleepMs = replicaMs; 377 } 378 379 @Override 380 protected RpcRetryingCaller<AbstractResponse> createCaller( 381 CancellableRegionServerCallable payloadCallable, int rpcTimeout) { 382 MultiServerCallable callable = (MultiServerCallable) payloadCallable; 383 final MultiResponse mr = createMultiResponse( 384 callable.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() { 385 @Override 386 public void addResponse(MultiResponse mr, byte[] regionName, Action a) { 387 if (failures.contains(regionName)) { 388 mr.add(regionName, a.getOriginalIndex(), failure); 389 } else { 390 boolean isStale = !RegionReplicaUtil.isDefaultReplica(a.getReplicaId()); 391 mr.add(regionName, a.getOriginalIndex(), 392 Result.create(new Cell[0], null, isStale)); 393 } 394 } 395 }); 396 // Currently AsyncProcess either sends all-replica, or all-primary request. 397 final boolean isDefault = RegionReplicaUtil.isDefaultReplica( 398 callable.getMulti().actions.values().iterator().next().iterator().next().getReplicaId()); 399 final ServerName server = ((MultiServerCallable)callable).getServerName(); 400 String debugMsg = "Call to " + server + ", primary=" + isDefault + " with " 401 + callable.getMulti().actions.size() + " entries: "; 402 for (byte[] region : callable.getMulti().actions.keySet()) { 403 debugMsg += "[" + Bytes.toStringBinary(region) + "], "; 404 } 405 LOG.debug(debugMsg); 406 if (!isDefault) { 407 replicaCalls.incrementAndGet(); 408 } 409 410 return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 10, 9) { 411 @Override 412 public MultiResponse callWithoutRetries(RetryingCallable<AbstractResponse> callable, 413 int callTimeout) 414 throws IOException, RuntimeException { 415 long sleep = -1; 416 if (isDefault) { 417 Long customSleep = customPrimarySleepMs.get(server); 418 sleep = (customSleep == null ? primarySleepMs : customSleep.longValue()); 419 } else { 420 sleep = replicaSleepMs; 421 } 422 if (sleep != 0) { 423 try { 424 Thread.sleep(sleep); 425 } catch (InterruptedException e) { 426 } 427 } 428 return mr; 429 } 430 }; 431 } 432 } 433 434 static MultiResponse createMultiResponse(final MultiAction multi, 435 AtomicInteger nbMultiResponse, AtomicInteger nbActions, ResponseGenerator gen) { 436 final MultiResponse mr = new MultiResponse(); 437 nbMultiResponse.incrementAndGet(); 438 for (Map.Entry<byte[], List<Action>> entry : multi.actions.entrySet()) { 439 byte[] regionName = entry.getKey(); 440 for (Action a : entry.getValue()) { 441 nbActions.incrementAndGet(); 442 gen.addResponse(mr, regionName, a); 443 } 444 } 445 return mr; 446 } 447 448 private static interface ResponseGenerator { 449 void addResponse(final MultiResponse mr, byte[] regionName, Action a); 450 } 451 452 /** 453 * Returns our async process. 454 */ 455 static class MyConnectionImpl extends ConnectionImplementation { 456 public static class TestRegistry extends DoNothingAsyncRegistry { 457 458 public TestRegistry(Configuration conf) { 459 super(conf); 460 } 461 462 @Override 463 public CompletableFuture<String> getClusterId() { 464 return CompletableFuture.completedFuture("testClusterId"); 465 } 466 467 @Override 468 public CompletableFuture<Integer> getCurrentNrHRS() { 469 return CompletableFuture.completedFuture(1); 470 } 471 } 472 473 final AtomicInteger nbThreads = new AtomicInteger(0); 474 475 protected MyConnectionImpl(Configuration conf) throws IOException { 476 super(setupConf(conf), null, null); 477 } 478 479 private static Configuration setupConf(Configuration conf) { 480 conf.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, TestRegistry.class, 481 AsyncRegistry.class); 482 return conf; 483 } 484 485 @Override 486 public RegionLocations locateRegion(TableName tableName, 487 byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException { 488 return new RegionLocations(loc1); 489 } 490 491 @Override 492 public boolean hasCellBlockSupport() { 493 return false; 494 } 495 } 496 497 /** 498 * Returns our async process. 499 */ 500 static class MyConnectionImpl2 extends MyConnectionImpl { 501 List<HRegionLocation> hrl; 502 final boolean usedRegions[]; 503 504 protected MyConnectionImpl2(List<HRegionLocation> hrl) throws IOException { 505 super(CONF); 506 this.hrl = hrl; 507 this.usedRegions = new boolean[hrl.size()]; 508 } 509 510 @Override 511 public RegionLocations locateRegion(TableName tableName, 512 byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException { 513 int i = 0; 514 for (HRegionLocation hr : hrl){ 515 if (Arrays.equals(row, hr.getRegionInfo().getStartKey())) { 516 usedRegions[i] = true; 517 return new RegionLocations(hr); 518 } 519 i++; 520 } 521 return null; 522 } 523 } 524 @Test 525 public void testListRowAccess() { 526 int count = 10; 527 List<String> values = new LinkedList<>(); 528 for (int i = 0; i != count; ++i) { 529 values.add(String.valueOf(i)); 530 } 531 532 ListRowAccess<String> taker = new ListRowAccess(values); 533 assertEquals(count, taker.size()); 534 535 int restoreCount = 0; 536 int takeCount = 0; 537 Iterator<String> it = taker.iterator(); 538 while (it.hasNext()) { 539 String v = it.next(); 540 assertEquals(String.valueOf(takeCount), v); 541 ++takeCount; 542 it.remove(); 543 if (Math.random() >= 0.5) { 544 break; 545 } 546 } 547 assertEquals(count, taker.size() + takeCount); 548 549 it = taker.iterator(); 550 while (it.hasNext()) { 551 String v = it.next(); 552 assertEquals(String.valueOf(takeCount), v); 553 ++takeCount; 554 it.remove(); 555 } 556 assertEquals(0, taker.size()); 557 assertEquals(count, takeCount); 558 } 559 private static long calculateRequestCount(long putSizePerServer, long maxHeapSizePerRequest) { 560 if (putSizePerServer <= maxHeapSizePerRequest) { 561 return 1; 562 } else if (putSizePerServer % maxHeapSizePerRequest == 0) { 563 return putSizePerServer / maxHeapSizePerRequest; 564 } else { 565 return putSizePerServer / maxHeapSizePerRequest + 1; 566 } 567 } 568 569 @Test 570 public void testSubmitSameSizeOfRequest() throws Exception { 571 long writeBuffer = 2 * 1024 * 1024; 572 long putsHeapSize = writeBuffer; 573 doSubmitRequest(writeBuffer, putsHeapSize); 574 } 575 576 @Test 577 public void testSubmitLargeRequestWithUnlimitedSize() throws Exception { 578 long maxHeapSizePerRequest = Long.MAX_VALUE; 579 long putsHeapSize = 2 * 1024 * 1024; 580 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); 581 } 582 583 @Test 584 public void testSubmitRandomSizeRequest() throws Exception { 585 Random rn = new Random(); 586 final long limit = 10 * 1024 * 1024; 587 final int requestCount = 1 + (int) (rn.nextDouble() * 3); 588 long n = rn.nextLong(); 589 if (n < 0) { 590 n = -n; 591 } else if (n == 0) { 592 n = 1; 593 } 594 long putsHeapSize = n % limit; 595 long maxHeapSizePerRequest = putsHeapSize / requestCount; 596 LOG.info("[testSubmitRandomSizeRequest] maxHeapSizePerRequest=" + maxHeapSizePerRequest + 597 ", putsHeapSize=" + putsHeapSize); 598 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); 599 } 600 601 @Test 602 public void testSubmitSmallRequest() throws Exception { 603 long maxHeapSizePerRequest = 2 * 1024 * 1024; 604 long putsHeapSize = 100; 605 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); 606 } 607 608 @Test 609 public void testSubmitLargeRequest() throws Exception { 610 long maxHeapSizePerRequest = 2 * 1024 * 1024; 611 long putsHeapSize = maxHeapSizePerRequest * 2; 612 doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); 613 } 614 615 private void doSubmitRequest(long maxHeapSizePerRequest, long putsHeapSize) throws Exception { 616 ClusterConnection conn = createHConnection(); 617 final String defaultClazz = 618 conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); 619 final long defaultHeapSizePerRequest = conn.getConfiguration().getLong( 620 SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, 621 SimpleRequestController.DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE); 622 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 623 SimpleRequestController.class.getName()); 624 conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, 625 maxHeapSizePerRequest); 626 627 // sn has two regions 628 long putSizeSN = 0; 629 long putSizeSN2 = 0; 630 List<Put> puts = new ArrayList<>(); 631 while ((putSizeSN + putSizeSN2) <= putsHeapSize) { 632 Put put1 = new Put(DUMMY_BYTES_1); 633 put1.addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1); 634 Put put2 = new Put(DUMMY_BYTES_2); 635 put2.addColumn(DUMMY_BYTES_2, DUMMY_BYTES_2, DUMMY_BYTES_2); 636 Put put3 = new Put(DUMMY_BYTES_3); 637 put3.addColumn(DUMMY_BYTES_3, DUMMY_BYTES_3, DUMMY_BYTES_3); 638 putSizeSN += (put1.heapSize() + put2.heapSize()); 639 putSizeSN2 += put3.heapSize(); 640 puts.add(put1); 641 puts.add(put2); 642 puts.add(put3); 643 } 644 645 int minCountSnRequest = (int) calculateRequestCount(putSizeSN, maxHeapSizePerRequest); 646 int minCountSn2Request = (int) calculateRequestCount(putSizeSN2, maxHeapSizePerRequest); 647 LOG.info("Total put count:" + puts.size() + ", putSizeSN:"+ putSizeSN 648 + ", putSizeSN2:" + putSizeSN2 649 + ", maxHeapSizePerRequest:" + maxHeapSizePerRequest 650 + ", minCountSnRequest:" + minCountSnRequest 651 + ", minCountSn2Request:" + minCountSn2Request); 652 653 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 654 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 655 try (BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap)) { 656 mutator.mutate(puts); 657 mutator.flush(); 658 List<AsyncRequestFuture> reqs = ap.allReqs; 659 660 int actualSnReqCount = 0; 661 int actualSn2ReqCount = 0; 662 for (AsyncRequestFuture req : reqs) { 663 if (!(req instanceof AsyncRequestFutureImpl)) { 664 continue; 665 } 666 MyAsyncRequestFutureImpl ars = (MyAsyncRequestFutureImpl) req; 667 if (ars.getRequestHeapSize().containsKey(sn)) { 668 ++actualSnReqCount; 669 } 670 if (ars.getRequestHeapSize().containsKey(sn2)) { 671 ++actualSn2ReqCount; 672 } 673 } 674 // If the server is busy, the actual count may be incremented. 675 assertEquals(true, minCountSnRequest <= actualSnReqCount); 676 assertEquals(true, minCountSn2Request <= actualSn2ReqCount); 677 Map<ServerName, Long> sizePerServers = new HashMap<>(); 678 for (AsyncRequestFuture req : reqs) { 679 if (!(req instanceof AsyncRequestFutureImpl)) { 680 continue; 681 } 682 MyAsyncRequestFutureImpl ars = (MyAsyncRequestFutureImpl) req; 683 Map<ServerName, List<Long>> requestHeapSize = ars.getRequestHeapSize(); 684 for (Map.Entry<ServerName, List<Long>> entry : requestHeapSize.entrySet()) { 685 long sum = 0; 686 for (long size : entry.getValue()) { 687 assertEquals(true, size <= maxHeapSizePerRequest); 688 sum += size; 689 } 690 assertEquals(true, sum <= maxHeapSizePerRequest); 691 long value = sizePerServers.getOrDefault(entry.getKey(), 0L); 692 sizePerServers.put(entry.getKey(), value + sum); 693 } 694 } 695 assertEquals(true, sizePerServers.containsKey(sn)); 696 assertEquals(true, sizePerServers.containsKey(sn2)); 697 assertEquals(false, sizePerServers.containsKey(sn3)); 698 assertEquals(putSizeSN, (long) sizePerServers.get(sn)); 699 assertEquals(putSizeSN2, (long) sizePerServers.get(sn2)); 700 } 701 // restore config. 702 conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, 703 defaultHeapSizePerRequest); 704 if (defaultClazz != null) { 705 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 706 defaultClazz); 707 } 708 } 709 710 @Test 711 public void testSubmit() throws Exception { 712 ClusterConnection hc = createHConnection(); 713 MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); 714 715 List<Put> puts = new ArrayList<>(1); 716 puts.add(createPut(1, true)); 717 718 ap.submit(null, DUMMY_TABLE, puts, false, null, false); 719 Assert.assertTrue(puts.isEmpty()); 720 } 721 722 @Test 723 public void testSubmitWithCB() throws Exception { 724 ClusterConnection hc = createHConnection(); 725 final AtomicInteger updateCalled = new AtomicInteger(0); 726 Batch.Callback<Object> cb = new Batch.Callback<Object>() { 727 @Override 728 public void update(byte[] region, byte[] row, Object result) { 729 updateCalled.incrementAndGet(); 730 } 731 }; 732 MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); 733 734 List<Put> puts = new ArrayList<>(1); 735 puts.add(createPut(1, true)); 736 737 final AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, cb, false); 738 Assert.assertTrue(puts.isEmpty()); 739 ars.waitUntilDone(); 740 Assert.assertEquals(1, updateCalled.get()); 741 } 742 743 @Test 744 public void testSubmitBusyRegion() throws Exception { 745 ClusterConnection conn = createHConnection(); 746 final String defaultClazz = 747 conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); 748 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 749 SimpleRequestController.class.getName()); 750 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 751 SimpleRequestController controller = (SimpleRequestController) ap.requestController; 752 List<Put> puts = new ArrayList<>(1); 753 puts.add(createPut(1, true)); 754 755 for (int i = 0; i != controller.maxConcurrentTasksPerRegion; ++i) { 756 ap.incTaskCounters(Collections.singleton(hri1.getRegionName()), sn); 757 } 758 ap.submit(null, DUMMY_TABLE, puts, false, null, false); 759 Assert.assertEquals(puts.size(), 1); 760 761 ap.decTaskCounters(Collections.singleton(hri1.getRegionName()), sn); 762 ap.submit(null, DUMMY_TABLE, puts, false, null, false); 763 Assert.assertEquals(0, puts.size()); 764 if (defaultClazz != null) { 765 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 766 defaultClazz); 767 } 768 } 769 770 771 @Test 772 public void testSubmitBusyRegionServer() throws Exception { 773 ClusterConnection conn = createHConnection(); 774 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 775 final String defaultClazz = 776 conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); 777 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 778 SimpleRequestController.class.getName()); 779 SimpleRequestController controller = (SimpleRequestController) ap.requestController; 780 controller.taskCounterPerServer.put(sn2, 781 new AtomicInteger(controller.maxConcurrentTasksPerServer)); 782 783 List<Put> puts = new ArrayList<>(4); 784 puts.add(createPut(1, true)); 785 puts.add(createPut(3, true)); // <== this one won't be taken, the rs is busy 786 puts.add(createPut(1, true)); // <== this one will make it, the region is already in 787 puts.add(createPut(2, true)); // <== new region, but the rs is ok 788 789 ap.submit(null, DUMMY_TABLE, puts, false, null, false); 790 Assert.assertEquals(" puts=" + puts, 1, puts.size()); 791 792 controller.taskCounterPerServer.put(sn2, 793 new AtomicInteger(controller.maxConcurrentTasksPerServer - 1)); 794 ap.submit(null, DUMMY_TABLE, puts, false, null, false); 795 Assert.assertTrue(puts.isEmpty()); 796 if (defaultClazz != null) { 797 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 798 defaultClazz); 799 } 800 } 801 802 @Test 803 public void testFail() throws Exception { 804 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF); 805 806 List<Put> puts = new ArrayList<>(1); 807 Put p = createPut(1, false); 808 puts.add(p); 809 810 AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); 811 Assert.assertEquals(0, puts.size()); 812 ars.waitUntilDone(); 813 verifyResult(ars, false); 814 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); 815 816 Assert.assertEquals(1, ars.getErrors().exceptions.size()); 817 Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0), 818 failure.equals(ars.getErrors().exceptions.get(0))); 819 Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0), 820 failure.equals(ars.getErrors().exceptions.get(0))); 821 822 Assert.assertEquals(1, ars.getFailedOperations().size()); 823 Assert.assertTrue("was: " + ars.getFailedOperations().get(0), 824 p.equals(ars.getFailedOperations().get(0))); 825 } 826 827 828 @Test 829 public void testSubmitTrue() throws IOException { 830 ClusterConnection conn = createHConnection(); 831 final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 832 final String defaultClazz = 833 conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); 834 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 835 SimpleRequestController.class.getName()); 836 SimpleRequestController controller = (SimpleRequestController) ap.requestController; 837 controller.tasksInProgress.incrementAndGet(); 838 final AtomicInteger ai = new AtomicInteger(controller.maxConcurrentTasksPerRegion); 839 controller.taskCounterPerRegion.put(hri1.getRegionName(), ai); 840 841 final AtomicBoolean checkPoint = new AtomicBoolean(false); 842 final AtomicBoolean checkPoint2 = new AtomicBoolean(false); 843 844 Thread t = new Thread(){ 845 @Override 846 public void run(){ 847 Threads.sleep(1000); 848 Assert.assertFalse(checkPoint.get()); // TODO: this is timing-dependent 849 ai.decrementAndGet(); 850 controller.tasksInProgress.decrementAndGet(); 851 checkPoint2.set(true); 852 } 853 }; 854 855 List<Put> puts = new ArrayList<>(1); 856 Put p = createPut(1, true); 857 puts.add(p); 858 859 ap.submit(null, DUMMY_TABLE, puts, false, null, false); 860 Assert.assertFalse(puts.isEmpty()); 861 862 t.start(); 863 864 ap.submit(null, DUMMY_TABLE, puts, true, null, false); 865 Assert.assertTrue(puts.isEmpty()); 866 867 checkPoint.set(true); 868 while (!checkPoint2.get()){ 869 Threads.sleep(1); 870 } 871 if (defaultClazz != null) { 872 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 873 defaultClazz); 874 } 875 } 876 877 @Test 878 public void testFailAndSuccess() throws Exception { 879 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF); 880 881 List<Put> puts = new ArrayList<>(3); 882 puts.add(createPut(1, false)); 883 puts.add(createPut(1, true)); 884 puts.add(createPut(1, true)); 885 886 AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); 887 Assert.assertTrue(puts.isEmpty()); 888 ars.waitUntilDone(); 889 verifyResult(ars, false, true, true); 890 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); 891 ap.callsCt.set(0); 892 Assert.assertEquals(1, ars.getErrors().actions.size()); 893 894 puts.add(createPut(1, true)); 895 // Wait for AP to be free. While ars might have the result, ap counters are decreased later. 896 ap.waitForMaximumCurrentTasks(0, null); 897 ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); 898 Assert.assertEquals(0, puts.size()); 899 ars.waitUntilDone(); 900 Assert.assertEquals(1, ap.callsCt.get()); 901 verifyResult(ars, true); 902 } 903 904 @Test 905 public void testFlush() throws Exception { 906 MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF); 907 908 List<Put> puts = new ArrayList<>(3); 909 puts.add(createPut(1, false)); 910 puts.add(createPut(1, true)); 911 puts.add(createPut(1, true)); 912 913 AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); 914 ars.waitUntilDone(); 915 verifyResult(ars, false, true, true); 916 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); 917 918 Assert.assertEquals(1, ars.getFailedOperations().size()); 919 } 920 921 @Test 922 public void testTaskCountWithoutClientBackoffPolicy() throws IOException, InterruptedException { 923 ClusterConnection hc = createHConnection(); 924 MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); 925 testTaskCount(ap); 926 } 927 928 @Test 929 public void testTaskCountWithClientBackoffPolicy() throws IOException, InterruptedException { 930 Configuration copyConf = new Configuration(CONF); 931 copyConf.setBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, true); 932 MyClientBackoffPolicy bp = new MyClientBackoffPolicy(); 933 ClusterConnection conn = createHConnection(); 934 Mockito.when(conn.getConfiguration()).thenReturn(copyConf); 935 Mockito.when(conn.getStatisticsTracker()).thenReturn(ServerStatisticTracker.create(copyConf)); 936 Mockito.when(conn.getBackoffPolicy()).thenReturn(bp); 937 final String defaultClazz = 938 conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); 939 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 940 SimpleRequestController.class.getName()); 941 MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf); 942 testTaskCount(ap); 943 if (defaultClazz != null) { 944 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 945 defaultClazz); 946 } 947 } 948 949 private void testTaskCount(MyAsyncProcess ap) 950 throws InterruptedIOException, InterruptedException { 951 SimpleRequestController controller = (SimpleRequestController) ap.requestController; 952 List<Put> puts = new ArrayList<>(); 953 for (int i = 0; i != 3; ++i) { 954 puts.add(createPut(1, true)); 955 puts.add(createPut(2, true)); 956 puts.add(createPut(3, true)); 957 } 958 ap.submit(null, DUMMY_TABLE, puts, true, null, false); 959 ap.waitForMaximumCurrentTasks(0, null); 960 // More time to wait if there are incorrect task count. 961 TimeUnit.SECONDS.sleep(1); 962 assertEquals(0, controller.tasksInProgress.get()); 963 for (AtomicInteger count : controller.taskCounterPerRegion.values()) { 964 assertEquals(0, count.get()); 965 } 966 for (AtomicInteger count : controller.taskCounterPerServer.values()) { 967 assertEquals(0, count.get()); 968 } 969 } 970 971 @Test 972 public void testMaxTask() throws Exception { 973 ClusterConnection conn = createHConnection(); 974 final String defaultClazz = 975 conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); 976 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 977 SimpleRequestController.class.getName()); 978 final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 979 SimpleRequestController controller = (SimpleRequestController) ap.requestController; 980 981 982 for (int i = 0; i < 1000; i++) { 983 ap.incTaskCounters(Collections.singleton(Bytes.toBytes("dummy")), sn); 984 } 985 986 final Thread myThread = Thread.currentThread(); 987 988 Thread t = new Thread() { 989 @Override 990 public void run() { 991 Threads.sleep(2000); 992 myThread.interrupt(); 993 } 994 }; 995 996 List<Put> puts = new ArrayList<>(1); 997 puts.add(createPut(1, true)); 998 999 t.start(); 1000 1001 try { 1002 ap.submit(null, DUMMY_TABLE, puts, false, null, false); 1003 Assert.fail("We should have been interrupted."); 1004 } catch (InterruptedIOException expected) { 1005 } 1006 1007 final long sleepTime = 2000; 1008 1009 Thread t2 = new Thread() { 1010 @Override 1011 public void run() { 1012 Threads.sleep(sleepTime); 1013 while (controller.tasksInProgress.get() > 0) { 1014 ap.decTaskCounters(Collections.singleton(Bytes.toBytes("dummy")), sn); 1015 } 1016 } 1017 }; 1018 t2.start(); 1019 1020 long start = System.currentTimeMillis(); 1021 ap.submit(null, DUMMY_TABLE, new ArrayList<>(), false, null, false); 1022 long end = System.currentTimeMillis(); 1023 1024 //Adds 100 to secure us against approximate timing. 1025 Assert.assertTrue(start + 100L + sleepTime > end); 1026 if (defaultClazz != null) { 1027 conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, 1028 defaultClazz); 1029 } 1030 } 1031 1032 private static ClusterConnection createHConnection() throws IOException { 1033 ClusterConnection hc = createHConnectionCommon(); 1034 setMockLocation(hc, DUMMY_BYTES_1, new RegionLocations(loc1)); 1035 setMockLocation(hc, DUMMY_BYTES_2, new RegionLocations(loc2)); 1036 setMockLocation(hc, DUMMY_BYTES_3, new RegionLocations(loc3)); 1037 Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), 1038 Mockito.anyBoolean())).thenReturn(Arrays.asList(loc1, loc2, loc3)); 1039 setMockLocation(hc, FAILS, new RegionLocations(loc2)); 1040 return hc; 1041 } 1042 1043 private static ClusterConnection createHConnectionWithReplicas() throws IOException { 1044 ClusterConnection hc = createHConnectionCommon(); 1045 setMockLocation(hc, DUMMY_BYTES_1, hrls1); 1046 setMockLocation(hc, DUMMY_BYTES_2, hrls2); 1047 setMockLocation(hc, DUMMY_BYTES_3, hrls3); 1048 List<HRegionLocation> locations = new ArrayList<>(); 1049 for (HRegionLocation loc : hrls1.getRegionLocations()) { 1050 locations.add(loc); 1051 } 1052 for (HRegionLocation loc : hrls2.getRegionLocations()) { 1053 locations.add(loc); 1054 } 1055 for (HRegionLocation loc : hrls3.getRegionLocations()) { 1056 locations.add(loc); 1057 } 1058 Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), 1059 Mockito.anyBoolean())).thenReturn(locations); 1060 return hc; 1061 } 1062 1063 private static void setMockLocation(ClusterConnection hc, byte[] row, 1064 RegionLocations result) throws IOException { 1065 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), 1066 Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result); 1067 Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), 1068 Mockito.anyBoolean(), Mockito.anyBoolean())).thenReturn(result); 1069 } 1070 1071 private static ClusterConnection createHConnectionCommon() { 1072 ClusterConnection hc = Mockito.mock(ClusterConnection.class); 1073 NonceGenerator ng = Mockito.mock(NonceGenerator.class); 1074 Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE); 1075 Mockito.when(hc.getNonceGenerator()).thenReturn(ng); 1076 Mockito.when(hc.getConfiguration()).thenReturn(CONF); 1077 Mockito.when(hc.getConnectionConfiguration()).thenReturn(CONNECTION_CONFIG); 1078 return hc; 1079 } 1080 1081 @Test 1082 public void testHTablePutSuccess() throws Exception { 1083 ClusterConnection conn = createHConnection(); 1084 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 1085 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1086 BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, bufferParam, ap); 1087 1088 Put put = createPut(1, true); 1089 1090 Assert.assertEquals(conn.getConnectionConfiguration().getWriteBufferSize(), 1091 ht.getWriteBufferSize()); 1092 Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); 1093 ht.mutate(put); 1094 ht.flush(); 1095 Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); 1096 } 1097 1098 @Test 1099 public void testSettingWriteBufferPeriodicFlushParameters() throws Exception { 1100 ClusterConnection conn = createHConnection(); 1101 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 1102 1103 checkPeriodicFlushParameters(conn, ap, 1104 1234, 1234, 1105 1234, 1234); 1106 checkPeriodicFlushParameters(conn, ap, 1107 0, 0, 1108 0, BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS); 1109 checkPeriodicFlushParameters(conn, ap, 1110 -1234, 0, 1111 -1234, BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS); 1112 checkPeriodicFlushParameters(conn, ap, 1113 1, 1, 1114 1, BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS); 1115 } 1116 1117 private void checkPeriodicFlushParameters(ClusterConnection conn, 1118 MyAsyncProcess ap, 1119 long setTO, long expectTO, 1120 long setTT, long expectTT 1121 ) { 1122 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1123 1124 // The BufferedMutatorParams does nothing with the value 1125 bufferParam.setWriteBufferPeriodicFlushTimeoutMs(setTO); 1126 bufferParam.setWriteBufferPeriodicFlushTimerTickMs(setTT); 1127 Assert.assertEquals(setTO, bufferParam.getWriteBufferPeriodicFlushTimeoutMs()); 1128 Assert.assertEquals(setTT, bufferParam.getWriteBufferPeriodicFlushTimerTickMs()); 1129 1130 // The BufferedMutatorImpl corrects illegal values (indirect via BufferedMutatorParams) 1131 BufferedMutatorImpl ht1 = new BufferedMutatorImpl(conn, bufferParam, ap); 1132 Assert.assertEquals(expectTO, ht1.getWriteBufferPeriodicFlushTimeoutMs()); 1133 Assert.assertEquals(expectTT, ht1.getWriteBufferPeriodicFlushTimerTickMs()); 1134 1135 // The BufferedMutatorImpl corrects illegal values (direct via setter) 1136 BufferedMutatorImpl ht2 = 1137 new BufferedMutatorImpl(conn, createBufferedMutatorParams(ap, DUMMY_TABLE), ap); 1138 ht2.setWriteBufferPeriodicFlush(setTO, setTT); 1139 Assert.assertEquals(expectTO, ht2.getWriteBufferPeriodicFlushTimeoutMs()); 1140 Assert.assertEquals(expectTT, ht2.getWriteBufferPeriodicFlushTimerTickMs()); 1141 1142 } 1143 1144 @Test 1145 public void testWriteBufferPeriodicFlushTimeoutMs() throws Exception { 1146 ClusterConnection conn = createHConnection(); 1147 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 1148 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1149 1150 bufferParam.setWriteBufferPeriodicFlushTimeoutMs(1); // Flush ASAP 1151 bufferParam.setWriteBufferPeriodicFlushTimerTickMs(1); // Check every 100ms 1152 bufferParam.writeBufferSize(10000); // Write buffer set to much larger than the single record 1153 1154 BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, bufferParam, ap); 1155 1156 // Verify if BufferedMutator has the right settings. 1157 Assert.assertEquals(10000, ht.getWriteBufferSize()); 1158 Assert.assertEquals(1, ht.getWriteBufferPeriodicFlushTimeoutMs()); 1159 Assert.assertEquals(BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS, 1160 ht.getWriteBufferPeriodicFlushTimerTickMs()); 1161 1162 Put put = createPut(1, true); 1163 1164 Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes()); 1165 Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); 1166 1167 // ----- Insert, flush immediately, MUST NOT flush automatically 1168 ht.mutate(put); 1169 ht.flush(); 1170 1171 Thread.sleep(1000); 1172 Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes()); 1173 Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); 1174 1175 // ----- Insert, NO flush, MUST flush automatically 1176 ht.mutate(put); 1177 Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes()); 1178 Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0); 1179 1180 // The timerTick should fire every 100ms, so after twice that we must have 1181 // seen at least 1 tick and we should see an automatic flush 1182 Thread.sleep(200); 1183 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); 1184 Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); 1185 1186 // Ensure it does not flush twice 1187 Thread.sleep(200); 1188 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); 1189 Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); 1190 1191 // ----- DISABLE AUTO FLUSH, Insert, NO flush, MUST NOT flush automatically 1192 ht.disableWriteBufferPeriodicFlush(); 1193 ht.mutate(put); 1194 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); 1195 Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0); 1196 1197 // Wait for at least 1 timerTick, we should see NO flushes. 1198 Thread.sleep(200); 1199 Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); 1200 Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0); 1201 1202 // Reenable periodic flushing, a flush seems to take about 1 second 1203 // so we wait for 2 seconds and it should have finished the flush. 1204 ht.setWriteBufferPeriodicFlush(1, 100); 1205 Thread.sleep(2000); 1206 Assert.assertEquals(2, ht.getExecutedWriteBufferPeriodicFlushes()); 1207 Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); 1208 } 1209 1210 1211 @Test 1212 public void testBufferedMutatorImplWithSharedPool() throws Exception { 1213 ClusterConnection conn = createHConnection(); 1214 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 1215 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1216 BufferedMutator ht = new BufferedMutatorImpl(conn, bufferParam, ap); 1217 1218 ht.close(); 1219 assertFalse(ap.service.isShutdown()); 1220 } 1221 1222 @Test 1223 public void testFailedPutAndNewPut() throws Exception { 1224 ClusterConnection conn = createHConnection(); 1225 MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); 1226 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE) 1227 .writeBufferSize(0); 1228 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); 1229 1230 Put p = createPut(1, false); 1231 try { 1232 mutator.mutate(p); 1233 Assert.fail(); 1234 } catch (RetriesExhaustedWithDetailsException expected) { 1235 assertEquals(1, expected.getNumExceptions()); 1236 assertTrue(expected.getRow(0) == p); 1237 } 1238 // Let's do all the retries. 1239 ap.waitForMaximumCurrentTasks(0, null); 1240 Assert.assertEquals(0, mutator.size()); 1241 1242 // There is no global error so the new put should not fail 1243 mutator.mutate(createPut(1, true)); 1244 Assert.assertEquals("the put should not been inserted.", 0, mutator.size()); 1245 } 1246 1247 @SuppressWarnings("SelfComparison") 1248 @Test 1249 public void testAction() { 1250 Action action_0 = new Action(new Put(Bytes.toBytes("abc")), 10); 1251 Action action_1 = new Action(new Put(Bytes.toBytes("ccc")), 10); 1252 Action action_2 = new Action(new Put(Bytes.toBytes("ccc")), 10); 1253 Action action_3 = new Action(new Delete(Bytes.toBytes("ccc")), 10); 1254 assertFalse(action_0.equals(action_1)); 1255 assertTrue(action_0.equals(action_0)); 1256 assertTrue(action_1.equals(action_2)); 1257 assertTrue(action_2.equals(action_1)); 1258 assertFalse(action_0.equals(new Put(Bytes.toBytes("abc")))); 1259 assertTrue(action_2.equals(action_3)); 1260 assertFalse(action_0.equals(action_3)); 1261 assertEquals(0, action_0.compareTo(action_0)); 1262 assertTrue(action_0.compareTo(action_1) < 0); 1263 assertTrue(action_1.compareTo(action_0) > 0); 1264 assertEquals(0, action_1.compareTo(action_2)); 1265 } 1266 1267 @Test 1268 public void testBatch() throws IOException, InterruptedException { 1269 ClusterConnection conn = new MyConnectionImpl(CONF); 1270 HTable ht = (HTable) conn.getTable(DUMMY_TABLE); 1271 ht.multiAp = new MyAsyncProcess(conn, CONF); 1272 1273 List<Put> puts = new ArrayList<>(7); 1274 puts.add(createPut(1, true)); 1275 puts.add(createPut(1, true)); 1276 puts.add(createPut(1, true)); 1277 puts.add(createPut(1, true)); 1278 puts.add(createPut(1, false)); // <=== the bad apple, position 4 1279 puts.add(createPut(1, true)); 1280 puts.add(createPut(1, false)); // <=== another bad apple, position 6 1281 1282 Object[] res = new Object[puts.size()]; 1283 try { 1284 ht.batch(puts, res); 1285 Assert.fail(); 1286 } catch (RetriesExhaustedException expected) { 1287 } 1288 1289 Assert.assertEquals(success, res[0]); 1290 Assert.assertEquals(success, res[1]); 1291 Assert.assertEquals(success, res[2]); 1292 Assert.assertEquals(success, res[3]); 1293 Assert.assertEquals(failure, res[4]); 1294 Assert.assertEquals(success, res[5]); 1295 Assert.assertEquals(failure, res[6]); 1296 } 1297 @Test 1298 public void testErrorsServers() throws IOException { 1299 Configuration configuration = new Configuration(CONF); 1300 ClusterConnection conn = new MyConnectionImpl(configuration); 1301 MyAsyncProcess ap = new MyAsyncProcess(conn, configuration); 1302 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1303 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); 1304 configuration.setBoolean(ConnectionImplementation.RETRIES_BY_SERVER_KEY, true); 1305 1306 Assert.assertNotNull(ap.createServerErrorTracker()); 1307 Assert.assertTrue(ap.serverTrackerTimeout > 200L); 1308 ap.serverTrackerTimeout = 1L; 1309 1310 Put p = createPut(1, false); 1311 mutator.mutate(p); 1312 1313 try { 1314 mutator.flush(); 1315 Assert.fail(); 1316 } catch (RetriesExhaustedWithDetailsException expected) { 1317 assertEquals(1, expected.getNumExceptions()); 1318 assertTrue(expected.getRow(0) == p); 1319 } 1320 // Checking that the ErrorsServers came into play and didn't make us stop immediately 1321 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); 1322 } 1323 1324 @Test 1325 public void testReadAndWriteTimeout() throws IOException { 1326 final long readTimeout = 10 * 1000; 1327 final long writeTimeout = 20 * 1000; 1328 Configuration copyConf = new Configuration(CONF); 1329 copyConf.setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, readTimeout); 1330 copyConf.setLong(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, writeTimeout); 1331 ClusterConnection conn = new MyConnectionImpl(copyConf); 1332 MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf); 1333 try (HTable ht = (HTable) conn.getTable(DUMMY_TABLE)) { 1334 ht.multiAp = ap; 1335 List<Get> gets = new LinkedList<>(); 1336 gets.add(new Get(DUMMY_BYTES_1)); 1337 gets.add(new Get(DUMMY_BYTES_2)); 1338 try { 1339 ht.get(gets); 1340 } catch (ClassCastException e) { 1341 // No result response on this test. 1342 } 1343 assertEquals(readTimeout, ap.previousTimeout); 1344 ap.previousTimeout = -1; 1345 1346 try { 1347 ht.existsAll(gets); 1348 } catch (ClassCastException e) { 1349 // No result response on this test. 1350 } 1351 assertEquals(readTimeout, ap.previousTimeout); 1352 ap.previousTimeout = -1; 1353 1354 List<Delete> deletes = new LinkedList<>(); 1355 deletes.add(new Delete(DUMMY_BYTES_1)); 1356 deletes.add(new Delete(DUMMY_BYTES_2)); 1357 ht.delete(deletes); 1358 assertEquals(writeTimeout, ap.previousTimeout); 1359 } 1360 } 1361 1362 @Test 1363 public void testErrors() throws IOException { 1364 ClusterConnection conn = new MyConnectionImpl(CONF); 1365 AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, CONF, new IOException("test")); 1366 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1367 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); 1368 1369 Assert.assertNotNull(ap.createServerErrorTracker()); 1370 1371 Put p = createPut(1, true); 1372 mutator.mutate(p); 1373 1374 try { 1375 mutator.flush(); 1376 Assert.fail(); 1377 } catch (RetriesExhaustedWithDetailsException expected) { 1378 assertEquals(1, expected.getNumExceptions()); 1379 assertTrue(expected.getRow(0) == p); 1380 } 1381 // Checking that the ErrorsServers came into play and didn't make us stop immediately 1382 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); 1383 } 1384 1385 1386 @Test 1387 public void testCallQueueTooLarge() throws IOException { 1388 ClusterConnection conn = new MyConnectionImpl(CONF); 1389 AsyncProcessWithFailure ap = 1390 new AsyncProcessWithFailure(conn, CONF, new CallQueueTooBigException()); 1391 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1392 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); 1393 Assert.assertNotNull(ap.createServerErrorTracker()); 1394 Put p = createPut(1, true); 1395 mutator.mutate(p); 1396 1397 try { 1398 mutator.flush(); 1399 Assert.fail(); 1400 } catch (RetriesExhaustedWithDetailsException expected) { 1401 assertEquals(1, expected.getNumExceptions()); 1402 assertTrue(expected.getRow(0) == p); 1403 } 1404 // Checking that the ErrorsServers came into play and didn't make us stop immediately 1405 Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); 1406 } 1407 /** 1408 * This test simulates multiple regions on 2 servers. We should have 2 multi requests and 1409 * 2 threads: 1 per server, this whatever the number of regions. 1410 */ 1411 @Test 1412 public void testThreadCreation() throws Exception { 1413 final int NB_REGS = 100; 1414 List<HRegionLocation> hrls = new ArrayList<>(NB_REGS); 1415 List<Get> gets = new ArrayList<>(NB_REGS); 1416 for (int i = 0; i < NB_REGS; i++) { 1417 HRegionInfo hri = new HRegionInfo( 1418 DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i); 1419 HRegionLocation hrl = new HRegionLocation(hri, i % 2 == 0 ? sn : sn2); 1420 hrls.add(hrl); 1421 1422 Get get = new Get(Bytes.toBytes(i * 10L)); 1423 gets.add(get); 1424 } 1425 1426 MyConnectionImpl2 con = new MyConnectionImpl2(hrls); 1427 MyAsyncProcess ap = new MyAsyncProcess(con, CONF, con.nbThreads); 1428 HTable ht = (HTable) con.getTable(DUMMY_TABLE, ap.service); 1429 ht.multiAp = ap; 1430 ht.batch(gets, null); 1431 1432 Assert.assertEquals(NB_REGS, ap.nbActions.get()); 1433 Assert.assertEquals("1 multi response per server", 2, ap.nbMultiResponse.get()); 1434 Assert.assertEquals("1 thread per server", 2, con.nbThreads.get()); 1435 1436 int nbReg = 0; 1437 for (int i =0; i<NB_REGS; i++){ 1438 if (con.usedRegions[i]) nbReg++; 1439 } 1440 Assert.assertEquals("nbReg=" + nbReg, NB_REGS, nbReg); 1441 } 1442 1443 @Test 1444 public void testReplicaReplicaSuccess() throws Exception { 1445 // Main call takes too long so replicas succeed, except for one region w/o replicas. 1446 // One region has no replica, so the main call succeeds for it. 1447 MyAsyncProcessWithReplicas ap = createReplicaAp(10, 1000, 0); 1448 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3); 1449 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1450 .setPool(ap.service) 1451 .setRpcTimeout(RPC_TIMEOUT) 1452 .setOperationTimeout(OPERATION_TIMEOUT) 1453 .setTableName(DUMMY_TABLE) 1454 .setRowAccess(rows) 1455 .setResults(new Object[3]) 1456 .setSubmittedRows(SubmittedRows.ALL) 1457 .build(); 1458 AsyncRequestFuture ars = ap.submit(task); 1459 verifyReplicaResult(ars, RR.TRUE, RR.TRUE, RR.FALSE); 1460 Assert.assertEquals(2, ap.getReplicaCallCount()); 1461 } 1462 1463 @Test 1464 public void testReplicaPrimarySuccessWoReplicaCalls() throws Exception { 1465 // Main call succeeds before replica calls are kicked off. 1466 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 10, 0); 1467 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3); 1468 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1469 .setPool(ap.service) 1470 .setRpcTimeout(RPC_TIMEOUT) 1471 .setOperationTimeout(OPERATION_TIMEOUT) 1472 .setTableName(DUMMY_TABLE) 1473 .setRowAccess(rows) 1474 .setResults(new Object[3]) 1475 .setSubmittedRows(SubmittedRows.ALL) 1476 .build(); 1477 AsyncRequestFuture ars = ap.submit(task); 1478 verifyReplicaResult(ars, RR.FALSE, RR.FALSE, RR.FALSE); 1479 Assert.assertEquals(0, ap.getReplicaCallCount()); 1480 } 1481 1482 @Test 1483 public void testReplicaParallelCallsSucceed() throws Exception { 1484 // Either main or replica can succeed. 1485 MyAsyncProcessWithReplicas ap = createReplicaAp(0, 0, 0); 1486 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); 1487 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1488 .setPool(ap.service) 1489 .setRpcTimeout(RPC_TIMEOUT) 1490 .setOperationTimeout(OPERATION_TIMEOUT) 1491 .setTableName(DUMMY_TABLE) 1492 .setRowAccess(rows) 1493 .setResults(new Object[2]) 1494 .setSubmittedRows(SubmittedRows.ALL) 1495 .build(); 1496 AsyncRequestFuture ars = ap.submit(task); 1497 verifyReplicaResult(ars, RR.DONT_CARE, RR.DONT_CARE); 1498 long replicaCalls = ap.getReplicaCallCount(); 1499 Assert.assertTrue(replicaCalls >= 0); 1500 Assert.assertTrue(replicaCalls <= 2); 1501 } 1502 1503 @Test 1504 public void testReplicaPartialReplicaCall() throws Exception { 1505 // One server is slow, so the result for its region comes from replica, whereas 1506 // the result for other region comes from primary before replica calls happen. 1507 // There should be no replica call for that region at all. 1508 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0); 1509 ap.setPrimaryCallDelay(sn2, 2000); 1510 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); 1511 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1512 .setPool(ap.service) 1513 .setRpcTimeout(RPC_TIMEOUT) 1514 .setOperationTimeout(OPERATION_TIMEOUT) 1515 .setTableName(DUMMY_TABLE) 1516 .setRowAccess(rows) 1517 .setResults(new Object[2]) 1518 .setSubmittedRows(SubmittedRows.ALL) 1519 .build(); 1520 AsyncRequestFuture ars = ap.submit(task); 1521 verifyReplicaResult(ars, RR.FALSE, RR.TRUE); 1522 Assert.assertEquals(1, ap.getReplicaCallCount()); 1523 } 1524 1525 @Test 1526 public void testReplicaMainFailsBeforeReplicaCalls() throws Exception { 1527 // Main calls fail before replica calls can start - this is currently not handled. 1528 // It would probably never happen if we can get location (due to retries), 1529 // and it would require additional synchronization. 1530 MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0, 0); 1531 ap.addFailures(hri1, hri2); 1532 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); 1533 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1534 .setPool(ap.service) 1535 .setRpcTimeout(RPC_TIMEOUT) 1536 .setOperationTimeout(OPERATION_TIMEOUT) 1537 .setTableName(DUMMY_TABLE) 1538 .setRowAccess(rows) 1539 .setResults(new Object[2]) 1540 .setSubmittedRows(SubmittedRows.ALL) 1541 .build(); 1542 AsyncRequestFuture ars = ap.submit(task); 1543 verifyReplicaResult(ars, RR.FAILED, RR.FAILED); 1544 Assert.assertEquals(0, ap.getReplicaCallCount()); 1545 } 1546 1547 @Test 1548 public void testReplicaReplicaSuccessWithParallelFailures() throws Exception { 1549 // Main calls fails after replica calls start. For two-replica region, one replica call 1550 // also fails. Regardless, we get replica results for both regions. 1551 MyAsyncProcessWithReplicas ap = createReplicaAp(0, 1000, 1000, 0); 1552 ap.addFailures(hri1, hri1r2, hri2); 1553 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); 1554 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1555 .setPool(ap.service) 1556 .setRpcTimeout(RPC_TIMEOUT) 1557 .setOperationTimeout(OPERATION_TIMEOUT) 1558 .setTableName(DUMMY_TABLE) 1559 .setRowAccess(rows) 1560 .setResults(new Object[2]) 1561 .setSubmittedRows(SubmittedRows.ALL) 1562 .build(); 1563 AsyncRequestFuture ars = ap.submit(task); 1564 verifyReplicaResult(ars, RR.TRUE, RR.TRUE); 1565 Assert.assertEquals(2, ap.getReplicaCallCount()); 1566 } 1567 1568 @Test 1569 public void testReplicaAllCallsFailForOneRegion() throws Exception { 1570 // For one of the region, all 3, main and replica, calls fail. For the other, replica 1571 // call fails but its exception should not be visible as it did succeed. 1572 MyAsyncProcessWithReplicas ap = createReplicaAp(500, 1000, 0, 0); 1573 ap.addFailures(hri1, hri1r1, hri1r2, hri2r1); 1574 List<Get> rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); 1575 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1576 .setPool(ap.service) 1577 .setRpcTimeout(RPC_TIMEOUT) 1578 .setOperationTimeout(OPERATION_TIMEOUT) 1579 .setTableName(DUMMY_TABLE) 1580 .setRowAccess(rows) 1581 .setResults(new Object[2]) 1582 .setSubmittedRows(SubmittedRows.ALL) 1583 .build(); 1584 AsyncRequestFuture ars = ap.submit(task); 1585 verifyReplicaResult(ars, RR.FAILED, RR.FALSE); 1586 // We should get 3 exceptions, for main + 2 replicas for DUMMY_BYTES_1 1587 Assert.assertEquals(3, ars.getErrors().getNumExceptions()); 1588 for (int i = 0; i < ars.getErrors().getNumExceptions(); ++i) { 1589 Assert.assertArrayEquals(DUMMY_BYTES_1, ars.getErrors().getRow(i).getRow()); 1590 } 1591 } 1592 1593 private MyAsyncProcessWithReplicas createReplicaAp( 1594 int replicaAfterMs, int primaryMs, int replicaMs) throws Exception { 1595 return createReplicaAp(replicaAfterMs, primaryMs, replicaMs, -1); 1596 } 1597 1598 private MyAsyncProcessWithReplicas createReplicaAp( 1599 int replicaAfterMs, int primaryMs, int replicaMs, int retries) throws Exception { 1600 // TODO: this is kind of timing dependent... perhaps it should detect from createCaller 1601 // that the replica call has happened and that way control the ordering. 1602 Configuration conf = new Configuration(); 1603 ClusterConnection conn = createHConnectionWithReplicas(); 1604 conf.setInt(AsyncProcess.PRIMARY_CALL_TIMEOUT_KEY, replicaAfterMs * 1000); 1605 if (retries >= 0) { 1606 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries); 1607 } 1608 MyAsyncProcessWithReplicas ap = new MyAsyncProcessWithReplicas(conn, conf); 1609 ap.setCallDelays(primaryMs, replicaMs); 1610 return ap; 1611 } 1612 1613 private static BufferedMutatorParams createBufferedMutatorParams(MyAsyncProcess ap, 1614 TableName name) { 1615 return new BufferedMutatorParams(name) 1616 .pool(ap.service) 1617 .rpcTimeout(RPC_TIMEOUT) 1618 .opertationTimeout(OPERATION_TIMEOUT); 1619 } 1620 1621 private static List<Get> makeTimelineGets(byte[]... rows) { 1622 List<Get> result = new ArrayList<>(rows.length); 1623 for (byte[] row : rows) { 1624 Get get = new Get(row); 1625 get.setConsistency(Consistency.TIMELINE); 1626 result.add(get); 1627 } 1628 return result; 1629 } 1630 1631 private void verifyResult(AsyncRequestFuture ars, boolean... expected) throws Exception { 1632 Object[] actual = ars.getResults(); 1633 Assert.assertEquals(expected.length, actual.length); 1634 for (int i = 0; i < expected.length; ++i) { 1635 Assert.assertEquals(expected[i], !(actual[i] instanceof Throwable)); 1636 } 1637 } 1638 1639 /** After reading TheDailyWtf, I always wanted to create a MyBoolean enum like this! */ 1640 private enum RR { 1641 TRUE, 1642 FALSE, 1643 DONT_CARE, 1644 FAILED 1645 } 1646 1647 private void verifyReplicaResult(AsyncRequestFuture ars, RR... expecteds) throws Exception { 1648 Object[] actuals = ars.getResults(); 1649 Assert.assertEquals(expecteds.length, actuals.length); 1650 for (int i = 0; i < expecteds.length; ++i) { 1651 Object actual = actuals[i]; 1652 RR expected = expecteds[i]; 1653 Assert.assertEquals(actual.toString(), expected == RR.FAILED, actual instanceof Throwable); 1654 if (expected != RR.FAILED && expected != RR.DONT_CARE) { 1655 Assert.assertEquals(expected == RR.TRUE, ((Result)actual).isStale()); 1656 } 1657 } 1658 } 1659 1660 /** 1661 * @param regCnt the region: 1 to 3. 1662 * @param success if true, the put will succeed. 1663 * @return a put 1664 */ 1665 private Put createPut(int regCnt, boolean success) { 1666 Put p; 1667 if (!success) { 1668 p = new Put(FAILS); 1669 } else switch (regCnt){ 1670 case 1 : 1671 p = new Put(DUMMY_BYTES_1); 1672 break; 1673 case 2: 1674 p = new Put(DUMMY_BYTES_2); 1675 break; 1676 case 3: 1677 p = new Put(DUMMY_BYTES_3); 1678 break; 1679 default: 1680 throw new IllegalArgumentException("unknown " + regCnt); 1681 } 1682 1683 p.addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1); 1684 1685 return p; 1686 } 1687 1688 static class MyThreadPoolExecutor extends ThreadPoolExecutor { 1689 public MyThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime, 1690 TimeUnit timeunit, BlockingQueue<Runnable> blockingqueue) { 1691 super(coreThreads, maxThreads, keepAliveTime, timeunit, blockingqueue); 1692 } 1693 1694 @Override 1695 public Future submit(Runnable runnable) { 1696 throw new OutOfMemoryError("OutOfMemory error thrown by means"); 1697 } 1698 } 1699 1700 static class AsyncProcessForThrowableCheck extends AsyncProcess { 1701 public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf) { 1702 super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory( 1703 conf)); 1704 } 1705 } 1706 1707 @Test 1708 public void testUncheckedException() throws Exception { 1709 // Test the case pool.submit throws unchecked exception 1710 ClusterConnection hc = createHConnection(); 1711 MyThreadPoolExecutor myPool = 1712 new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, 1713 new LinkedBlockingQueue<>(200)); 1714 AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, CONF); 1715 1716 List<Put> puts = new ArrayList<>(1); 1717 puts.add(createPut(1, true)); 1718 AsyncProcessTask task = AsyncProcessTask.newBuilder() 1719 .setPool(myPool) 1720 .setRpcTimeout(RPC_TIMEOUT) 1721 .setOperationTimeout(OPERATION_TIMEOUT) 1722 .setTableName(DUMMY_TABLE) 1723 .setRowAccess(puts) 1724 .setSubmittedRows(SubmittedRows.NORMAL) 1725 .build(); 1726 ap.submit(task); 1727 Assert.assertTrue(puts.isEmpty()); 1728 } 1729 1730 /** 1731 * Test and make sure we could use a special pause setting when retry with 1732 * CallQueueTooBigException, see HBASE-17114 1733 * @throws Exception if unexpected error happened during test 1734 */ 1735 @Test 1736 public void testRetryPauseWithCallQueueTooBigException() throws Exception { 1737 Configuration myConf = new Configuration(CONF); 1738 final long specialPause = 500L; 1739 final int retries = 1; 1740 myConf.setLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, specialPause); 1741 myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries); 1742 ClusterConnection conn = new MyConnectionImpl(myConf); 1743 AsyncProcessWithFailure ap = 1744 new AsyncProcessWithFailure(conn, myConf, new CallQueueTooBigException()); 1745 BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1746 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); 1747 1748 Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker()); 1749 1750 Put p = createPut(1, true); 1751 mutator.mutate(p); 1752 1753 long startTime = System.currentTimeMillis(); 1754 try { 1755 mutator.flush(); 1756 Assert.fail(); 1757 } catch (RetriesExhaustedWithDetailsException expected) { 1758 assertEquals(1, expected.getNumExceptions()); 1759 assertTrue(expected.getRow(0) == p); 1760 } 1761 long actualSleep = System.currentTimeMillis() - startTime; 1762 long expectedSleep = 0L; 1763 for (int i = 0; i < retries; i++) { 1764 expectedSleep += ConnectionUtils.getPauseTime(specialPause, i); 1765 // Prevent jitter in CollectionUtils#getPauseTime to affect result 1766 actualSleep += (long) (specialPause * 0.01f); 1767 } 1768 LOG.debug("Expected to sleep " + expectedSleep + "ms, actually slept " + actualSleep + "ms"); 1769 Assert.assertTrue("Expected to sleep " + expectedSleep + " but actually " + actualSleep + "ms", 1770 actualSleep >= expectedSleep); 1771 1772 // check and confirm normal IOE will use the normal pause 1773 final long normalPause = 1774 myConf.getLong(HConstants.HBASE_CLIENT_PAUSE, HConstants.DEFAULT_HBASE_CLIENT_PAUSE); 1775 ap = new AsyncProcessWithFailure(conn, myConf, new IOException()); 1776 bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); 1777 mutator = new BufferedMutatorImpl(conn, bufferParam, ap); 1778 Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker()); 1779 mutator.mutate(p); 1780 startTime = System.currentTimeMillis(); 1781 try { 1782 mutator.flush(); 1783 Assert.fail(); 1784 } catch (RetriesExhaustedWithDetailsException expected) { 1785 assertEquals(1, expected.getNumExceptions()); 1786 assertTrue(expected.getRow(0) == p); 1787 } 1788 actualSleep = System.currentTimeMillis() - startTime; 1789 expectedSleep = 0L; 1790 for (int i = 0; i < retries; i++) { 1791 expectedSleep += ConnectionUtils.getPauseTime(normalPause, i); 1792 } 1793 // plus an additional pause to balance the program execution time 1794 expectedSleep += normalPause; 1795 LOG.debug("Expected to sleep " + expectedSleep + "ms, actually slept " + actualSleep + "ms"); 1796 Assert.assertTrue("Slept for too long: " + actualSleep + "ms", actualSleep <= expectedSleep); 1797 } 1798 1799 @Test 1800 public void testQueueRowAccess() throws Exception { 1801 ClusterConnection conn = createHConnection(); 1802 BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null, 1803 new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(100000)); 1804 Put p0 = new Put(DUMMY_BYTES_1).addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1); 1805 Put p1 = new Put(DUMMY_BYTES_2).addColumn(DUMMY_BYTES_2, DUMMY_BYTES_2, DUMMY_BYTES_2); 1806 mutator.mutate(p0); 1807 BufferedMutatorImpl.QueueRowAccess ra0 = mutator.createQueueRowAccess(); 1808 // QueueRowAccess should take all undealt mutations 1809 assertEquals(0, mutator.size()); 1810 mutator.mutate(p1); 1811 assertEquals(1, mutator.size()); 1812 BufferedMutatorImpl.QueueRowAccess ra1 = mutator.createQueueRowAccess(); 1813 // QueueRowAccess should take all undealt mutations 1814 assertEquals(0, mutator.size()); 1815 assertEquals(1, ra0.size()); 1816 assertEquals(1, ra1.size()); 1817 Iterator<Row> iter0 = ra0.iterator(); 1818 Iterator<Row> iter1 = ra1.iterator(); 1819 assertTrue(iter0.hasNext()); 1820 assertTrue(iter1.hasNext()); 1821 // the next() will poll the mutation from inner buffer and update the buffer count 1822 assertTrue(iter0.next() == p0); 1823 assertEquals(1, mutator.getUnflushedSize()); 1824 assertEquals(p1.heapSize(), mutator.getCurrentWriteBufferSize()); 1825 assertTrue(iter1.next() == p1); 1826 assertEquals(0, mutator.getUnflushedSize()); 1827 assertEquals(0, mutator.getCurrentWriteBufferSize()); 1828 assertFalse(iter0.hasNext()); 1829 assertFalse(iter1.hasNext()); 1830 // ra0 doest handle the mutation so the mutation won't be pushed back to buffer 1831 iter0.remove(); 1832 ra0.close(); 1833 assertEquals(0, mutator.size()); 1834 assertEquals(0, mutator.getUnflushedSize()); 1835 assertEquals(0, mutator.getCurrentWriteBufferSize()); 1836 // ra1 doesn't handle the mutation so the mutation will be pushed back to buffer 1837 ra1.close(); 1838 assertEquals(1, mutator.size()); 1839 assertEquals(1, mutator.getUnflushedSize()); 1840 assertEquals(p1.heapSize(), mutator.getCurrentWriteBufferSize()); 1841 } 1842}