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 java.io.IOException; 021import java.util.AbstractMap.SimpleEntry; 022import java.util.ArrayList; 023import java.util.Collections; 024import java.util.HashMap; 025import java.util.List; 026import java.util.Map; 027import java.util.concurrent.ConcurrentHashMap; 028import java.util.concurrent.ExecutorService; 029import java.util.concurrent.Executors; 030import java.util.concurrent.LinkedBlockingQueue; 031import java.util.concurrent.ScheduledExecutorService; 032import java.util.concurrent.TimeUnit; 033import java.util.concurrent.atomic.AtomicInteger; 034import java.util.concurrent.atomic.AtomicLong; 035import org.apache.hadoop.conf.Configuration; 036import org.apache.hadoop.hbase.HBaseConfiguration; 037import org.apache.hadoop.hbase.HConstants; 038import org.apache.hadoop.hbase.HRegionLocation; 039import org.apache.hadoop.hbase.ServerName; 040import org.apache.hadoop.hbase.TableName; 041import org.apache.hadoop.hbase.ipc.RpcControllerFactory; 042import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 043import org.apache.yetus.audience.InterfaceAudience; 044import org.slf4j.Logger; 045import org.slf4j.LoggerFactory; 046 047import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 048 049/** 050 * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables. Each put 051 * will be sharded into different buffer queues based on its destination region server. So each 052 * region server buffer queue will only have the puts which share the same destination. And each 053 * queue will have a flush worker thread to flush the puts request to the region server. If any 054 * queue is full, the HTableMultiplexer starts to drop the Put requests for that particular queue. 055 * </p> 056 * Also all the puts will be retried as a configuration number before dropping. And the 057 * HTableMultiplexer can report the number of buffered requests and the number of the failed 058 * (dropped) requests in total or on per region server basis. 059 * <p/> 060 * This class is thread safe. 061 * @deprecated since 2.2.0, will be removed in 3.0.0, without replacement. Please use 062 * {@link BufferedMutator} for batching mutations. 063 */ 064@Deprecated 065@InterfaceAudience.Public 066public class HTableMultiplexer { 067 private static final Logger LOG = LoggerFactory.getLogger(HTableMultiplexer.class.getName()); 068 069 public static final String TABLE_MULTIPLEXER_FLUSH_PERIOD_MS = 070 "hbase.tablemultiplexer.flush.period.ms"; 071 public static final String TABLE_MULTIPLEXER_INIT_THREADS = "hbase.tablemultiplexer.init.threads"; 072 public static final String TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE = 073 "hbase.client.max.retries.in.queue"; 074 075 /** The map between each region server to its flush worker */ 076 private final Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap = 077 new ConcurrentHashMap<>(); 078 079 private final Configuration workerConf; 080 private final ClusterConnection conn; 081 private final ExecutorService pool; 082 private final int maxAttempts; 083 private final int perRegionServerBufferQueueSize; 084 private final int maxKeyValueSize; 085 private final ScheduledExecutorService executor; 086 private final long flushPeriod; 087 088 /** 089 * @param conf The HBaseConfiguration 090 * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for 091 * each region server before dropping the request. 092 */ 093 public HTableMultiplexer(Configuration conf, int perRegionServerBufferQueueSize) 094 throws IOException { 095 this(ConnectionFactory.createConnection(conf), conf, perRegionServerBufferQueueSize); 096 } 097 098 /** 099 * @param conn The HBase connection. 100 * @param conf The HBase configuration 101 * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for 102 * each region server before dropping the request. 103 */ 104 public HTableMultiplexer(Connection conn, Configuration conf, 105 int perRegionServerBufferQueueSize) { 106 this.conn = (ClusterConnection) conn; 107 this.pool = HTable.getDefaultExecutor(conf); 108 // how many times we could try in total, one more than retry number 109 this.maxAttempts = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 110 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER) + 1; 111 this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize; 112 this.maxKeyValueSize = HTable.getMaxKeyValueSize(conf); 113 this.flushPeriod = conf.getLong(TABLE_MULTIPLEXER_FLUSH_PERIOD_MS, 100); 114 int initThreads = conf.getInt(TABLE_MULTIPLEXER_INIT_THREADS, 10); 115 this.executor = Executors.newScheduledThreadPool(initThreads, 116 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("HTableFlushWorker-%d").build()); 117 118 this.workerConf = HBaseConfiguration.create(conf); 119 // We do not do the retry because we need to reassign puts to different queues if regions are 120 // moved. 121 this.workerConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0); 122 } 123 124 /** 125 * Closes the internal {@link Connection}. Does nothing if the {@link Connection} has already been 126 * closed. 127 * @throws IOException If there is an error closing the connection. 128 */ 129 public synchronized void close() throws IOException { 130 if (!getConnection().isClosed()) { 131 getConnection().close(); 132 } 133 } 134 135 /** 136 * The put request will be buffered by its corresponding buffer queue. Return false if the queue 137 * is already full. 138 * @return true if the request can be accepted by its corresponding buffer queue. 139 */ 140 public boolean put(TableName tableName, final Put put) { 141 return put(tableName, put, this.maxAttempts); 142 } 143 144 /** 145 * The puts request will be buffered by their corresponding buffer queue. Return the list of puts 146 * which could not be queued. 147 * @return the list of puts which could not be queued 148 */ 149 public List<Put> put(TableName tableName, final List<Put> puts) { 150 if (puts == null) return null; 151 152 List<Put> failedPuts = null; 153 boolean result; 154 for (Put put : puts) { 155 result = put(tableName, put, this.maxAttempts); 156 if (result == false) { 157 158 // Create the failed puts list if necessary 159 if (failedPuts == null) { 160 failedPuts = new ArrayList<>(); 161 } 162 // Add the put to the failed puts list 163 failedPuts.add(put); 164 } 165 } 166 return failedPuts; 167 } 168 169 /** 170 * @deprecated Use {@link #put(TableName, List) } instead. 171 */ 172 @Deprecated 173 public List<Put> put(byte[] tableName, final List<Put> puts) { 174 return put(TableName.valueOf(tableName), puts); 175 } 176 177 /** 178 * The put request will be buffered by its corresponding buffer queue. And the put request will be 179 * retried before dropping the request. Return false if the queue is already full. 180 * @return true if the request can be accepted by its corresponding buffer queue. 181 */ 182 public boolean put(final TableName tableName, final Put put, int maxAttempts) { 183 if (maxAttempts <= 0) { 184 return false; 185 } 186 187 try { 188 ConnectionUtils.validatePut(put, maxKeyValueSize); 189 // Allow mocking to get at the connection, but don't expose the connection to users. 190 ClusterConnection conn = (ClusterConnection) getConnection(); 191 // AsyncProcess in the FlushWorker should take care of refreshing the location cache 192 // as necessary. We shouldn't have to do that here. 193 HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false); 194 if (loc != null) { 195 // Add the put pair into its corresponding queue. 196 LinkedBlockingQueue<PutStatus> queue = getQueue(loc); 197 198 // Generate a MultiPutStatus object and offer it into the queue 199 PutStatus s = new PutStatus(loc.getRegion(), put, maxAttempts); 200 201 return queue.offer(s); 202 } 203 } catch (IOException e) { 204 LOG.debug("Cannot process the put " + put, e); 205 } 206 return false; 207 } 208 209 /** 210 * @deprecated Use {@link #put(TableName, Put) } instead. 211 */ 212 @Deprecated 213 public boolean put(final byte[] tableName, final Put put, int retry) { 214 return put(TableName.valueOf(tableName), put, retry); 215 } 216 217 /** 218 * @deprecated Use {@link #put(TableName, Put)} instead. 219 */ 220 @Deprecated 221 public boolean put(final byte[] tableName, Put put) { 222 return put(TableName.valueOf(tableName), put); 223 } 224 225 /** Returns the current HTableMultiplexerStatus */ 226 public HTableMultiplexerStatus getHTableMultiplexerStatus() { 227 return new HTableMultiplexerStatus(serverToFlushWorkerMap); 228 } 229 230 @InterfaceAudience.Private 231 LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) { 232 FlushWorker worker = serverToFlushWorkerMap.get(addr); 233 if (worker == null) { 234 synchronized (this.serverToFlushWorkerMap) { 235 worker = serverToFlushWorkerMap.get(addr); 236 if (worker == null) { 237 // Create the flush worker 238 worker = new FlushWorker(workerConf, this.conn, addr, this, 239 perRegionServerBufferQueueSize, pool, executor); 240 this.serverToFlushWorkerMap.put(addr, worker); 241 executor.scheduleAtFixedRate(worker, flushPeriod, flushPeriod, TimeUnit.MILLISECONDS); 242 } 243 } 244 } 245 return worker.getQueue(); 246 } 247 248 @InterfaceAudience.Private 249 ClusterConnection getConnection() { 250 return this.conn; 251 } 252 253 /** 254 * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer. report the 255 * number of buffered requests and the number of the failed (dropped) requests in total or on per 256 * region server basis. 257 * @deprecated since 2.2.0, will be removed in 3.0.0, without replacement. Please use 258 * {@link BufferedMutator} for batching mutations. 259 */ 260 @Deprecated 261 @InterfaceAudience.Public 262 public static class HTableMultiplexerStatus { 263 private long totalFailedPutCounter; 264 private long totalBufferedPutCounter; 265 private long maxLatency; 266 private long overallAverageLatency; 267 private Map<String, Long> serverToFailedCounterMap; 268 private Map<String, Long> serverToBufferedCounterMap; 269 private Map<String, Long> serverToAverageLatencyMap; 270 private Map<String, Long> serverToMaxLatencyMap; 271 272 public HTableMultiplexerStatus(Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) { 273 this.totalBufferedPutCounter = 0; 274 this.totalFailedPutCounter = 0; 275 this.maxLatency = 0; 276 this.overallAverageLatency = 0; 277 this.serverToBufferedCounterMap = new HashMap<>(); 278 this.serverToFailedCounterMap = new HashMap<>(); 279 this.serverToAverageLatencyMap = new HashMap<>(); 280 this.serverToMaxLatencyMap = new HashMap<>(); 281 this.initialize(serverToFlushWorkerMap); 282 } 283 284 private void initialize(Map<HRegionLocation, FlushWorker> serverToFlushWorkerMap) { 285 if (serverToFlushWorkerMap == null) { 286 return; 287 } 288 289 long averageCalcSum = 0; 290 int averageCalcCount = 0; 291 for (Map.Entry<HRegionLocation, FlushWorker> entry : serverToFlushWorkerMap.entrySet()) { 292 HRegionLocation addr = entry.getKey(); 293 FlushWorker worker = entry.getValue(); 294 295 long bufferedCounter = worker.getTotalBufferedCount(); 296 long failedCounter = worker.getTotalFailedCount(); 297 long serverMaxLatency = worker.getMaxLatency(); 298 AtomicAverageCounter averageCounter = worker.getAverageLatencyCounter(); 299 // Get sum and count pieces separately to compute overall average 300 SimpleEntry<Long, Integer> averageComponents = averageCounter.getComponents(); 301 long serverAvgLatency = averageCounter.getAndReset(); 302 303 this.totalBufferedPutCounter += bufferedCounter; 304 this.totalFailedPutCounter += failedCounter; 305 if (serverMaxLatency > this.maxLatency) { 306 this.maxLatency = serverMaxLatency; 307 } 308 averageCalcSum += averageComponents.getKey(); 309 averageCalcCount += averageComponents.getValue(); 310 311 this.serverToBufferedCounterMap.put(addr.getHostnamePort(), bufferedCounter); 312 this.serverToFailedCounterMap.put(addr.getHostnamePort(), failedCounter); 313 this.serverToAverageLatencyMap.put(addr.getHostnamePort(), serverAvgLatency); 314 this.serverToMaxLatencyMap.put(addr.getHostnamePort(), serverMaxLatency); 315 } 316 this.overallAverageLatency = averageCalcCount != 0 ? averageCalcSum / averageCalcCount : 0; 317 } 318 319 public long getTotalBufferedCounter() { 320 return this.totalBufferedPutCounter; 321 } 322 323 public long getTotalFailedCounter() { 324 return this.totalFailedPutCounter; 325 } 326 327 public long getMaxLatency() { 328 return this.maxLatency; 329 } 330 331 public long getOverallAverageLatency() { 332 return this.overallAverageLatency; 333 } 334 335 public Map<String, Long> getBufferedCounterForEachRegionServer() { 336 return this.serverToBufferedCounterMap; 337 } 338 339 public Map<String, Long> getFailedCounterForEachRegionServer() { 340 return this.serverToFailedCounterMap; 341 } 342 343 public Map<String, Long> getMaxLatencyForEachRegionServer() { 344 return this.serverToMaxLatencyMap; 345 } 346 347 public Map<String, Long> getAverageLatencyForEachRegionServer() { 348 return this.serverToAverageLatencyMap; 349 } 350 } 351 352 @InterfaceAudience.Private 353 static class PutStatus { 354 final RegionInfo regionInfo; 355 final Put put; 356 final int maxAttempCount; 357 358 public PutStatus(RegionInfo regionInfo, Put put, int maxAttempCount) { 359 this.regionInfo = regionInfo; 360 this.put = put; 361 this.maxAttempCount = maxAttempCount; 362 } 363 } 364 365 /** 366 * Helper to count the average over an interval until reset. 367 */ 368 private static class AtomicAverageCounter { 369 private long sum; 370 private int count; 371 372 public AtomicAverageCounter() { 373 this.sum = 0L; 374 this.count = 0; 375 } 376 377 public synchronized long getAndReset() { 378 long result = this.get(); 379 this.reset(); 380 return result; 381 } 382 383 public synchronized long get() { 384 if (this.count == 0) { 385 return 0; 386 } 387 return this.sum / this.count; 388 } 389 390 public synchronized SimpleEntry<Long, Integer> getComponents() { 391 return new SimpleEntry<>(sum, count); 392 } 393 394 public synchronized void reset() { 395 this.sum = 0L; 396 this.count = 0; 397 } 398 399 public synchronized void add(long value) { 400 this.sum += value; 401 this.count++; 402 } 403 } 404 405 @InterfaceAudience.Private 406 static class FlushWorker implements Runnable { 407 private final HRegionLocation addr; 408 private final LinkedBlockingQueue<PutStatus> queue; 409 private final HTableMultiplexer multiplexer; 410 private final AtomicLong totalFailedPutCount = new AtomicLong(0); 411 private final AtomicInteger currentProcessingCount = new AtomicInteger(0); 412 private final AtomicAverageCounter averageLatency = new AtomicAverageCounter(); 413 private final AtomicLong maxLatency = new AtomicLong(0); 414 415 private final AsyncProcess ap; 416 private final List<PutStatus> processingList = new ArrayList<>(); 417 private final ScheduledExecutorService executor; 418 private final int maxRetryInQueue; 419 private final AtomicInteger retryInQueue = new AtomicInteger(0); 420 private final int writeRpcTimeout; // needed to pass in through AsyncProcess constructor 421 private final int operationTimeout; 422 private final ExecutorService pool; 423 424 public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation addr, 425 HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize, ExecutorService pool, 426 ScheduledExecutorService executor) { 427 this.addr = addr; 428 this.multiplexer = htableMultiplexer; 429 this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize); 430 RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf); 431 RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf); 432 this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, 433 conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); 434 this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 435 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); 436 this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, rpcControllerFactory); 437 this.executor = executor; 438 this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000); 439 this.pool = pool; 440 } 441 442 protected LinkedBlockingQueue<PutStatus> getQueue() { 443 return this.queue; 444 } 445 446 public long getTotalFailedCount() { 447 return totalFailedPutCount.get(); 448 } 449 450 public long getTotalBufferedCount() { 451 return (long) queue.size() + currentProcessingCount.get(); 452 } 453 454 public AtomicAverageCounter getAverageLatencyCounter() { 455 return this.averageLatency; 456 } 457 458 public long getMaxLatency() { 459 return this.maxLatency.getAndSet(0); 460 } 461 462 boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException { 463 // Decrease the retry count 464 final int retryCount = ps.maxAttempCount - 1; 465 466 if (retryCount <= 0) { 467 // Update the failed counter and no retry any more. 468 return false; 469 } 470 471 int cnt = getRetryInQueue().incrementAndGet(); 472 if (cnt > getMaxRetryInQueue()) { 473 // Too many Puts in queue for resubmit, give up this 474 getRetryInQueue().decrementAndGet(); 475 return false; 476 } 477 478 final Put failedPut = ps.put; 479 // The currentPut is failed. So get the table name for the currentPut. 480 final TableName tableName = ps.regionInfo.getTable(); 481 482 long delayMs = getNextDelay(retryCount); 483 if (LOG.isDebugEnabled()) { 484 LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount); 485 } 486 487 // HBASE-12198, HBASE-15221, HBASE-15232: AsyncProcess should be responsible for updating 488 // the region location cache when the Put original failed with some exception. If we keep 489 // re-trying the same Put to the same location, AsyncProcess isn't doing the right stuff 490 // that we expect it to. 491 getExecutor().schedule(new Runnable() { 492 @Override 493 public void run() { 494 boolean succ = false; 495 try { 496 succ = FlushWorker.this.getMultiplexer().put(tableName, failedPut, retryCount); 497 } finally { 498 FlushWorker.this.getRetryInQueue().decrementAndGet(); 499 if (!succ) { 500 FlushWorker.this.getTotalFailedPutCount().incrementAndGet(); 501 } 502 } 503 } 504 }, delayMs, TimeUnit.MILLISECONDS); 505 return true; 506 } 507 508 @InterfaceAudience.Private 509 long getNextDelay(int retryCount) { 510 return ConnectionUtils.getPauseTime(multiplexer.flushPeriod, 511 multiplexer.maxAttempts - retryCount - 1); 512 } 513 514 @InterfaceAudience.Private 515 AtomicInteger getRetryInQueue() { 516 return this.retryInQueue; 517 } 518 519 @InterfaceAudience.Private 520 int getMaxRetryInQueue() { 521 return this.maxRetryInQueue; 522 } 523 524 @InterfaceAudience.Private 525 AtomicLong getTotalFailedPutCount() { 526 return this.totalFailedPutCount; 527 } 528 529 @InterfaceAudience.Private 530 HTableMultiplexer getMultiplexer() { 531 return this.multiplexer; 532 } 533 534 @InterfaceAudience.Private 535 ScheduledExecutorService getExecutor() { 536 return this.executor; 537 } 538 539 @Override 540 public void run() { 541 int failedCount = 0; 542 try { 543 long start = EnvironmentEdgeManager.currentTime(); 544 545 // drain all the queued puts into the tmp list 546 processingList.clear(); 547 queue.drainTo(processingList); 548 if (processingList.isEmpty()) { 549 // Nothing to flush 550 return; 551 } 552 553 currentProcessingCount.set(processingList.size()); 554 // failedCount is decreased whenever a Put is success or resubmit. 555 failedCount = processingList.size(); 556 557 List<Action> retainedActions = new ArrayList<>(processingList.size()); 558 MultiAction actions = new MultiAction(); 559 for (int i = 0; i < processingList.size(); i++) { 560 PutStatus putStatus = processingList.get(i); 561 Action action = new Action(putStatus.put, i); 562 actions.add(putStatus.regionInfo.getRegionName(), action); 563 retainedActions.add(action); 564 } 565 566 // Process this multi-put request 567 List<PutStatus> failed = null; 568 Object[] results = new Object[actions.size()]; 569 ServerName server = addr.getServerName(); 570 Map<ServerName, MultiAction> actionsByServer = Collections.singletonMap(server, actions); 571 try { 572 AsyncProcessTask task = AsyncProcessTask.newBuilder().setResults(results).setPool(pool) 573 .setRpcTimeout(writeRpcTimeout).setOperationTimeout(operationTimeout).build(); 574 AsyncRequestFuture arf = 575 ap.submitMultiActions(task, retainedActions, 0L, null, null, actionsByServer); 576 arf.waitUntilDone(); 577 if (arf.hasError()) { 578 // We just log and ignore the exception here since failed Puts will be resubmit again. 579 LOG.debug("Caught some exceptions when flushing puts to region server " 580 + addr.getHostnamePort(), arf.getErrors()); 581 } 582 } finally { 583 for (int i = 0; i < results.length; i++) { 584 if (results[i] instanceof Result) { 585 failedCount--; 586 } else { 587 if (failed == null) { 588 failed = new ArrayList<>(); 589 } 590 failed.add(processingList.get(i)); 591 } 592 } 593 } 594 595 if (failed != null) { 596 // Resubmit failed puts 597 for (PutStatus putStatus : failed) { 598 if (resubmitFailedPut(putStatus, this.addr)) { 599 failedCount--; 600 } 601 } 602 } 603 604 long elapsed = EnvironmentEdgeManager.currentTime() - start; 605 // Update latency counters 606 averageLatency.add(elapsed); 607 if (elapsed > maxLatency.get()) { 608 maxLatency.set(elapsed); 609 } 610 611 // Log some basic info 612 if (LOG.isDebugEnabled()) { 613 LOG.debug( 614 "Processed " + currentProcessingCount + " put requests for " + addr.getHostnamePort() 615 + " and " + failedCount + " failed" + ", latency for this send: " + elapsed); 616 } 617 618 // Reset the current processing put count 619 currentProcessingCount.set(0); 620 } catch (RuntimeException e) { 621 // To make findbugs happy 622 // Log all the exceptions and move on 623 LOG.debug("Caught some exceptions " + e + " when flushing puts to region server " 624 + addr.getHostnamePort(), e); 625 } catch (Exception e) { 626 if (e instanceof InterruptedException) { 627 Thread.currentThread().interrupt(); 628 } 629 // Log all the exceptions and move on 630 LOG.debug("Caught some exceptions " + e + " when flushing puts to region server " 631 + addr.getHostnamePort(), e); 632 } finally { 633 // Update the totalFailedCount 634 this.totalFailedPutCount.addAndGet(failedCount); 635 } 636 } 637 } 638}