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 */ 018 019package org.apache.hadoop.hbase.replication.regionserver; 020 021import java.io.IOException; 022import java.io.InterruptedIOException; 023import java.util.ArrayList; 024import java.util.List; 025import java.util.Map; 026import java.util.concurrent.Callable; 027import java.util.concurrent.ExecutionException; 028import java.util.concurrent.ExecutorService; 029import java.util.concurrent.Future; 030import java.util.concurrent.LinkedBlockingQueue; 031import java.util.concurrent.ThreadPoolExecutor; 032import java.util.concurrent.TimeUnit; 033import java.util.concurrent.atomic.AtomicLong; 034import org.apache.hadoop.conf.Configuration; 035import org.apache.hadoop.fs.Path; 036import org.apache.hadoop.hbase.CellScanner; 037import org.apache.hadoop.hbase.CellUtil; 038import org.apache.hadoop.hbase.HBaseConfiguration; 039import org.apache.hadoop.hbase.HBaseIOException; 040import org.apache.hadoop.hbase.HConstants; 041import org.apache.hadoop.hbase.HRegionLocation; 042import org.apache.hadoop.hbase.RegionLocations; 043import org.apache.hadoop.hbase.TableDescriptors; 044import org.apache.hadoop.hbase.TableName; 045import org.apache.hadoop.hbase.TableNotFoundException; 046import org.apache.hadoop.hbase.client.ClusterConnection; 047import org.apache.hadoop.hbase.client.ConnectionFactory; 048import org.apache.hadoop.hbase.client.RegionAdminServiceCallable; 049import org.apache.hadoop.hbase.client.RegionInfo; 050import org.apache.hadoop.hbase.client.RegionReplicaUtil; 051import org.apache.hadoop.hbase.client.RetryingCallable; 052import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; 053import org.apache.hadoop.hbase.client.TableDescriptor; 054import org.apache.hadoop.hbase.ipc.HBaseRpcController; 055import org.apache.hadoop.hbase.ipc.RpcControllerFactory; 056import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; 057import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint; 058import org.apache.hadoop.hbase.replication.WALEntryFilter; 059import org.apache.hadoop.hbase.util.Bytes; 060import org.apache.hadoop.hbase.util.Pair; 061import org.apache.hadoop.hbase.util.Threads; 062import org.apache.hadoop.hbase.wal.EntryBuffers; 063import org.apache.hadoop.hbase.wal.OutputSink; 064import org.apache.hadoop.hbase.wal.WAL.Entry; 065import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController; 066import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer; 067import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter; 068import org.apache.hadoop.util.StringUtils; 069import org.apache.yetus.audience.InterfaceAudience; 070import org.slf4j.Logger; 071import org.slf4j.LoggerFactory; 072 073import org.apache.hbase.thirdparty.com.google.common.cache.Cache; 074import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; 075 076import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; 077import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; 078 079/** 080 * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} endpoint 081 * which receives the WAL edits from the WAL, and sends the edits to replicas 082 * of regions. 083 */ 084@InterfaceAudience.Private 085public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { 086 087 private static final Logger LOG = LoggerFactory.getLogger(RegionReplicaReplicationEndpoint.class); 088 089 // Can be configured differently than hbase.client.retries.number 090 private static String CLIENT_RETRIES_NUMBER 091 = "hbase.region.replica.replication.client.retries.number"; 092 093 private Configuration conf; 094 private ClusterConnection connection; 095 private TableDescriptors tableDescriptors; 096 097 // Reuse WALSplitter constructs as a WAL pipe 098 private PipelineController controller; 099 private RegionReplicaOutputSink outputSink; 100 private EntryBuffers entryBuffers; 101 102 // Number of writer threads 103 private int numWriterThreads; 104 105 private int operationTimeout; 106 107 private ExecutorService pool; 108 109 @Override 110 public void init(Context context) throws IOException { 111 super.init(context); 112 113 this.conf = HBaseConfiguration.create(context.getConfiguration()); 114 this.tableDescriptors = context.getTableDescriptors(); 115 116 // HRS multiplies client retries by 10 globally for meta operations, but we do not want this. 117 // We are resetting it here because we want default number of retries (35) rather than 10 times 118 // that which makes very long retries for disabled tables etc. 119 int defaultNumRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 120 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); 121 if (defaultNumRetries > 10) { 122 int mult = conf.getInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER, 123 HConstants.DEFAULT_HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER); 124 defaultNumRetries = defaultNumRetries / mult; // reset if HRS has multiplied this already 125 } 126 127 conf.setInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER, 1); 128 int numRetries = conf.getInt(CLIENT_RETRIES_NUMBER, defaultNumRetries); 129 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries); 130 131 this.numWriterThreads = this.conf.getInt( 132 "hbase.region.replica.replication.writer.threads", 3); 133 controller = new PipelineController(); 134 entryBuffers = new EntryBuffers(controller, 135 this.conf.getLong("hbase.region.replica.replication.buffersize", 128 * 1024 * 1024)); 136 137 // use the regular RPC timeout for replica replication RPC's 138 this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 139 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); 140 } 141 142 @Override 143 protected void doStart() { 144 try { 145 connection = (ClusterConnection) ConnectionFactory.createConnection(this.conf); 146 this.pool = getDefaultThreadPool(conf); 147 outputSink = new RegionReplicaOutputSink(controller, tableDescriptors, entryBuffers, 148 connection, pool, numWriterThreads, operationTimeout); 149 outputSink.startWriterThreads(); 150 super.doStart(); 151 } catch (IOException ex) { 152 LOG.warn("Received exception while creating connection :" + ex); 153 notifyFailed(ex); 154 } 155 } 156 157 @Override 158 protected void doStop() { 159 if (outputSink != null) { 160 try { 161 outputSink.finishWritingAndClose(); 162 } catch (IOException ex) { 163 LOG.warn("Got exception while trying to close OutputSink", ex); 164 } 165 } 166 if (this.pool != null) { 167 this.pool.shutdownNow(); 168 try { 169 // wait for 10 sec 170 boolean shutdown = this.pool.awaitTermination(10000, TimeUnit.MILLISECONDS); 171 if (!shutdown) { 172 LOG.warn("Failed to shutdown the thread pool after 10 seconds"); 173 } 174 } catch (InterruptedException e) { 175 LOG.warn("Got interrupted while waiting for the thread pool to shut down" + e); 176 } 177 } 178 if (connection != null) { 179 try { 180 connection.close(); 181 } catch (IOException ex) { 182 LOG.warn("Got exception closing connection :" + ex); 183 } 184 } 185 super.doStop(); 186 } 187 188 /** 189 * Returns a Thread pool for the RPC's to region replicas. Similar to 190 * Connection's thread pool. 191 */ 192 private ExecutorService getDefaultThreadPool(Configuration conf) { 193 int maxThreads = conf.getInt("hbase.region.replica.replication.threads.max", 256); 194 if (maxThreads == 0) { 195 maxThreads = Runtime.getRuntime().availableProcessors() * 8; 196 } 197 long keepAliveTime = conf.getLong("hbase.region.replica.replication.threads.keepalivetime", 60); 198 LinkedBlockingQueue<Runnable> workQueue = 199 new LinkedBlockingQueue<>(maxThreads * 200 conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, 201 HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); 202 ThreadPoolExecutor tpe = new ThreadPoolExecutor( 203 maxThreads, 204 maxThreads, 205 keepAliveTime, 206 TimeUnit.SECONDS, 207 workQueue, 208 Threads.newDaemonThreadFactory(this.getClass().getSimpleName() + "-rpc-shared-")); 209 tpe.allowCoreThreadTimeOut(true); 210 return tpe; 211 } 212 213 @Override 214 public boolean replicate(ReplicateContext replicateContext) { 215 /* A note on batching in RegionReplicaReplicationEndpoint (RRRE): 216 * 217 * RRRE relies on batching from two different mechanisms. The first is the batching from 218 * ReplicationSource since RRRE is a ReplicationEndpoint driven by RS. RS reads from a single 219 * WAL file filling up a buffer of heap size "replication.source.size.capacity"(64MB) or at most 220 * "replication.source.nb.capacity" entries or until it sees the end of file (in live tailing). 221 * Then RS passes all the buffered edits in this replicate() call context. RRRE puts the edits 222 * to the WALSplitter.EntryBuffers which is a blocking buffer space of up to 223 * "hbase.region.replica.replication.buffersize" (128MB) in size. This buffer splits the edits 224 * based on regions. 225 * 226 * There are "hbase.region.replica.replication.writer.threads"(default 3) writer threads which 227 * pick largest per-region buffer and send it to the SinkWriter (see RegionReplicaOutputSink). 228 * The SinkWriter in this case will send the wal edits to all secondary region replicas in 229 * parallel via a retrying rpc call. EntryBuffers guarantees that while a buffer is 230 * being written to the sink, another buffer for the same region will not be made available to 231 * writers ensuring regions edits are not replayed out of order. 232 * 233 * The replicate() call won't return until all the buffers are sent and ack'd by the sinks so 234 * that the replication can assume all edits are persisted. We may be able to do a better 235 * pipelining between the replication thread and output sinks later if it becomes a bottleneck. 236 */ 237 238 while (this.isRunning()) { 239 try { 240 for (Entry entry: replicateContext.getEntries()) { 241 entryBuffers.appendEntry(entry); 242 } 243 outputSink.flush(); // make sure everything is flushed 244 ctx.getMetrics().incrLogEditsFiltered( 245 outputSink.getSkippedEditsCounter().getAndSet(0)); 246 return true; 247 } catch (InterruptedException e) { 248 Thread.currentThread().interrupt(); 249 return false; 250 } catch (IOException e) { 251 LOG.warn("Received IOException while trying to replicate" 252 + StringUtils.stringifyException(e)); 253 } 254 } 255 256 return false; 257 } 258 259 @Override 260 public boolean canReplicateToSameCluster() { 261 return true; 262 } 263 264 @Override 265 protected WALEntryFilter getScopeWALEntryFilter() { 266 // we do not care about scope. We replicate everything. 267 return null; 268 } 269 270 static class RegionReplicaOutputSink extends OutputSink { 271 private final RegionReplicaSinkWriter sinkWriter; 272 private final TableDescriptors tableDescriptors; 273 private final Cache<TableName, Boolean> memstoreReplicationEnabled; 274 275 public RegionReplicaOutputSink(PipelineController controller, TableDescriptors tableDescriptors, 276 EntryBuffers entryBuffers, ClusterConnection connection, ExecutorService pool, 277 int numWriters, int operationTimeout) { 278 super(controller, entryBuffers, numWriters); 279 this.sinkWriter = 280 new RegionReplicaSinkWriter(this, connection, pool, operationTimeout, tableDescriptors); 281 this.tableDescriptors = tableDescriptors; 282 283 // A cache for the table "memstore replication enabled" flag. 284 // It has a default expiry of 5 sec. This means that if the table is altered 285 // with a different flag value, we might miss to replicate for that amount of 286 // time. But this cache avoid the slow lookup and parsing of the TableDescriptor. 287 int memstoreReplicationEnabledCacheExpiryMs = connection.getConfiguration() 288 .getInt("hbase.region.replica.replication.cache.memstoreReplicationEnabled.expiryMs", 5000); 289 this.memstoreReplicationEnabled = CacheBuilder.newBuilder() 290 .expireAfterWrite(memstoreReplicationEnabledCacheExpiryMs, TimeUnit.MILLISECONDS) 291 .initialCapacity(10) 292 .maximumSize(1000) 293 .build(); 294 } 295 296 @Override 297 public void append(RegionEntryBuffer buffer) throws IOException { 298 List<Entry> entries = buffer.getEntryBuffer(); 299 300 if (entries.isEmpty() || entries.get(0).getEdit().getCells().isEmpty()) { 301 return; 302 } 303 304 // meta edits (e.g. flush) are always replicated. 305 // data edits (e.g. put) are replicated if the table requires them. 306 if (!requiresReplication(buffer.getTableName(), entries)) { 307 return; 308 } 309 310 sinkWriter.append(buffer.getTableName(), buffer.getEncodedRegionName(), 311 CellUtil.cloneRow(entries.get(0).getEdit().getCells().get(0)), entries); 312 } 313 314 @Override 315 public boolean flush() throws IOException { 316 // nothing much to do for now. Wait for the Writer threads to finish up 317 // append()'ing the data. 318 entryBuffers.waitUntilDrained(); 319 return super.flush(); 320 } 321 322 @Override 323 public boolean keepRegionEvent(Entry entry) { 324 return true; 325 } 326 327 @Override 328 public List<Path> finishWritingAndClose() throws IOException { 329 finishWriting(true); 330 return null; 331 } 332 333 @Override 334 public Map<byte[], Long> getOutputCounts() { 335 return null; // only used in tests 336 } 337 338 @Override 339 public int getNumberOfRecoveredRegions() { 340 return 0; 341 } 342 343 AtomicLong getSkippedEditsCounter() { 344 return skippedEdits; 345 } 346 347 /** 348 * returns true if the specified entry must be replicated. 349 * We should always replicate meta operations (e.g. flush) 350 * and use the user HTD flag to decide whether or not replicate the memstore. 351 */ 352 private boolean requiresReplication(final TableName tableName, final List<Entry> entries) 353 throws IOException { 354 // unit-tests may not the TableDescriptors, bypass the check and always replicate 355 if (tableDescriptors == null) return true; 356 357 Boolean requiresReplication = memstoreReplicationEnabled.getIfPresent(tableName); 358 if (requiresReplication == null) { 359 // check if the table requires memstore replication 360 // some unit-test drop the table, so we should do a bypass check and always replicate. 361 TableDescriptor htd = tableDescriptors.get(tableName); 362 requiresReplication = htd == null || htd.hasRegionMemStoreReplication(); 363 memstoreReplicationEnabled.put(tableName, requiresReplication); 364 } 365 366 // if memstore replication is not required, check the entries. 367 // meta edits (e.g. flush) must be always replicated. 368 if (!requiresReplication) { 369 int skipEdits = 0; 370 java.util.Iterator<Entry> it = entries.iterator(); 371 while (it.hasNext()) { 372 Entry entry = it.next(); 373 if (entry.getEdit().isMetaEdit()) { 374 requiresReplication = true; 375 } else { 376 it.remove(); 377 skipEdits++; 378 } 379 } 380 skippedEdits.addAndGet(skipEdits); 381 } 382 return requiresReplication; 383 } 384 } 385 386 static class RegionReplicaSinkWriter extends SinkWriter { 387 RegionReplicaOutputSink sink; 388 ClusterConnection connection; 389 RpcControllerFactory rpcControllerFactory; 390 RpcRetryingCallerFactory rpcRetryingCallerFactory; 391 int operationTimeout; 392 ExecutorService pool; 393 Cache<TableName, Boolean> disabledAndDroppedTables; 394 TableDescriptors tableDescriptors; 395 396 public RegionReplicaSinkWriter(RegionReplicaOutputSink sink, ClusterConnection connection, 397 ExecutorService pool, int operationTimeout, TableDescriptors tableDescriptors) { 398 this.sink = sink; 399 this.connection = connection; 400 this.operationTimeout = operationTimeout; 401 this.rpcRetryingCallerFactory 402 = RpcRetryingCallerFactory.instantiate(connection.getConfiguration()); 403 this.rpcControllerFactory = RpcControllerFactory.instantiate(connection.getConfiguration()); 404 this.pool = pool; 405 this.tableDescriptors = tableDescriptors; 406 407 int nonExistentTableCacheExpiryMs = connection.getConfiguration() 408 .getInt("hbase.region.replica.replication.cache.disabledAndDroppedTables.expiryMs", 5000); 409 // A cache for non existing tables that have a default expiry of 5 sec. This means that if the 410 // table is created again with the same name, we might miss to replicate for that amount of 411 // time. But this cache prevents overloading meta requests for every edit from a deleted file. 412 disabledAndDroppedTables = CacheBuilder.newBuilder() 413 .expireAfterWrite(nonExistentTableCacheExpiryMs, TimeUnit.MILLISECONDS) 414 .initialCapacity(10) 415 .maximumSize(1000) 416 .build(); 417 } 418 419 public void append(TableName tableName, byte[] encodedRegionName, byte[] row, 420 List<Entry> entries) throws IOException { 421 422 if (disabledAndDroppedTables.getIfPresent(tableName) != null) { 423 if (LOG.isTraceEnabled()) { 424 LOG.trace("Skipping " + entries.size() + " entries because table " + tableName 425 + " is cached as a disabled or dropped table"); 426 for (Entry entry : entries) { 427 LOG.trace("Skipping : " + entry); 428 } 429 } 430 sink.getSkippedEditsCounter().addAndGet(entries.size()); 431 return; 432 } 433 434 // If the table is disabled or dropped, we should not replay the entries, and we can skip 435 // replaying them. However, we might not know whether the table is disabled until we 436 // invalidate the cache and check from meta 437 RegionLocations locations = null; 438 boolean useCache = true; 439 while (true) { 440 // get the replicas of the primary region 441 try { 442 locations = RegionReplicaReplayCallable 443 .getRegionLocations(connection, tableName, row, useCache, 0); 444 445 if (locations == null) { 446 throw new HBaseIOException("Cannot locate locations for " 447 + tableName + ", row:" + Bytes.toStringBinary(row)); 448 } 449 } catch (TableNotFoundException e) { 450 if (LOG.isTraceEnabled()) { 451 LOG.trace("Skipping " + entries.size() + " entries because table " + tableName 452 + " is dropped. Adding table to cache."); 453 for (Entry entry : entries) { 454 LOG.trace("Skipping : " + entry); 455 } 456 } 457 disabledAndDroppedTables.put(tableName, Boolean.TRUE); // put to cache. Value ignored 458 // skip this entry 459 sink.getSkippedEditsCounter().addAndGet(entries.size()); 460 return; 461 } 462 463 // check whether we should still replay this entry. If the regions are changed, or the 464 // entry is not coming from the primary region, filter it out. 465 HRegionLocation primaryLocation = locations.getDefaultRegionLocation(); 466 if (!Bytes.equals(primaryLocation.getRegionInfo().getEncodedNameAsBytes(), 467 encodedRegionName)) { 468 if (useCache) { 469 useCache = false; 470 continue; // this will retry location lookup 471 } 472 if (LOG.isTraceEnabled()) { 473 LOG.trace("Skipping " + entries.size() + " entries in table " + tableName 474 + " because located region " + primaryLocation.getRegionInfo().getEncodedName() 475 + " is different than the original region " + Bytes.toStringBinary(encodedRegionName) 476 + " from WALEdit"); 477 for (Entry entry : entries) { 478 LOG.trace("Skipping : " + entry); 479 } 480 } 481 sink.getSkippedEditsCounter().addAndGet(entries.size()); 482 return; 483 } 484 break; 485 } 486 487 if (locations.size() == 1) { 488 return; 489 } 490 491 ArrayList<Future<ReplicateWALEntryResponse>> tasks = new ArrayList<>(locations.size() - 1); 492 493 // All passed entries should belong to one region because it is coming from the EntryBuffers 494 // split per region. But the regions might split and merge (unlike log recovery case). 495 for (int replicaId = 0; replicaId < locations.size(); replicaId++) { 496 HRegionLocation location = locations.getRegionLocation(replicaId); 497 if (!RegionReplicaUtil.isDefaultReplica(replicaId)) { 498 RegionInfo regionInfo = location == null 499 ? RegionReplicaUtil.getRegionInfoForReplica( 500 locations.getDefaultRegionLocation().getRegionInfo(), replicaId) 501 : location.getRegionInfo(); 502 RegionReplicaReplayCallable callable = new RegionReplicaReplayCallable(connection, 503 rpcControllerFactory, tableName, location, regionInfo, row, entries, 504 sink.getSkippedEditsCounter()); 505 Future<ReplicateWALEntryResponse> task = pool.submit( 506 new RetryingRpcCallable<>(rpcRetryingCallerFactory, callable, operationTimeout)); 507 tasks.add(task); 508 } 509 } 510 511 boolean tasksCancelled = false; 512 for (int replicaId = 0; replicaId < tasks.size(); replicaId++) { 513 try { 514 tasks.get(replicaId).get(); 515 } catch (InterruptedException e) { 516 throw new InterruptedIOException(e.getMessage()); 517 } catch (ExecutionException e) { 518 Throwable cause = e.getCause(); 519 boolean canBeSkipped = false; 520 if (cause instanceof IOException) { 521 // The table can be disabled or dropped at this time. For disabled tables, we have no 522 // cheap mechanism to detect this case because meta does not contain this information. 523 // ClusterConnection.isTableDisabled() is a zk call which we cannot do for every replay 524 // RPC. So instead we start the replay RPC with retries and check whether the table is 525 // dropped or disabled which might cause SocketTimeoutException, or 526 // RetriesExhaustedException or similar if we get IOE. 527 if (cause instanceof TableNotFoundException 528 || connection.isTableDisabled(tableName)) { 529 disabledAndDroppedTables.put(tableName, Boolean.TRUE); // put to cache for later. 530 canBeSkipped = true; 531 } else if (tableDescriptors != null) { 532 TableDescriptor tableDescriptor = tableDescriptors.get(tableName); 533 if (tableDescriptor != null 534 //(replicaId + 1) as no task is added for primary replica for replication 535 && tableDescriptor.getRegionReplication() <= (replicaId + 1)) { 536 canBeSkipped = true; 537 } 538 } 539 if (canBeSkipped) { 540 if (LOG.isTraceEnabled()) { 541 LOG.trace("Skipping " + entries.size() + " entries in table " + tableName 542 + " because received exception for dropped or disabled table", 543 cause); 544 for (Entry entry : entries) { 545 LOG.trace("Skipping : " + entry); 546 } 547 } 548 if (!tasksCancelled) { 549 sink.getSkippedEditsCounter().addAndGet(entries.size()); 550 tasksCancelled = true; // so that we do not add to skipped counter again 551 } 552 continue; 553 } 554 555 // otherwise rethrow 556 throw (IOException)cause; 557 } 558 // unexpected exception 559 throw new IOException(cause); 560 } 561 } 562 } 563 } 564 565 static class RetryingRpcCallable<V> implements Callable<V> { 566 RpcRetryingCallerFactory factory; 567 RetryingCallable<V> callable; 568 int timeout; 569 public RetryingRpcCallable(RpcRetryingCallerFactory factory, RetryingCallable<V> callable, 570 int timeout) { 571 this.factory = factory; 572 this.callable = callable; 573 this.timeout = timeout; 574 } 575 @Override 576 public V call() throws Exception { 577 return factory.<V>newCaller().callWithRetries(callable, timeout); 578 } 579 } 580 581 /** 582 * Calls replay on the passed edits for the given set of entries belonging to the region. It skips 583 * the entry if the region boundaries have changed or the region is gone. 584 */ 585 static class RegionReplicaReplayCallable extends 586 RegionAdminServiceCallable<ReplicateWALEntryResponse> { 587 private final List<Entry> entries; 588 private final byte[] initialEncodedRegionName; 589 private final AtomicLong skippedEntries; 590 591 public RegionReplicaReplayCallable(ClusterConnection connection, 592 RpcControllerFactory rpcControllerFactory, TableName tableName, 593 HRegionLocation location, RegionInfo regionInfo, byte[] row,List<Entry> entries, 594 AtomicLong skippedEntries) { 595 super(connection, rpcControllerFactory, location, tableName, row, regionInfo.getReplicaId()); 596 this.entries = entries; 597 this.skippedEntries = skippedEntries; 598 this.initialEncodedRegionName = regionInfo.getEncodedNameAsBytes(); 599 } 600 601 @Override 602 public ReplicateWALEntryResponse call(HBaseRpcController controller) throws Exception { 603 // Check whether we should still replay this entry. If the regions are changed, or the 604 // entry is not coming form the primary region, filter it out because we do not need it. 605 // Regions can change because of (1) region split (2) region merge (3) table recreated 606 boolean skip = false; 607 if (!Bytes.equals(location.getRegionInfo().getEncodedNameAsBytes(), 608 initialEncodedRegionName)) { 609 skip = true; 610 } 611 if (!this.entries.isEmpty() && !skip) { 612 Entry[] entriesArray = new Entry[this.entries.size()]; 613 entriesArray = this.entries.toArray(entriesArray); 614 615 // set the region name for the target region replica 616 Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p = 617 ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location 618 .getRegionInfo().getEncodedNameAsBytes(), null, null, null); 619 controller.setCellScanner(p.getSecond()); 620 return stub.replay(controller, p.getFirst()); 621 } 622 623 if (skip) { 624 if (LOG.isTraceEnabled()) { 625 LOG.trace("Skipping " + entries.size() + " entries in table " + tableName 626 + " because located region " + location.getRegionInfo().getEncodedName() 627 + " is different than the original region " 628 + Bytes.toStringBinary(initialEncodedRegionName) + " from WALEdit"); 629 for (Entry entry : entries) { 630 LOG.trace("Skipping : " + entry); 631 } 632 } 633 skippedEntries.addAndGet(entries.size()); 634 } 635 return ReplicateWALEntryResponse.newBuilder().build(); 636 } 637 } 638}