Skip to content

Commit 3b0013d

Browse files
committed
HBASE-22731 ReplicationSource and HBaseInterClusterReplicationEndpoint log messages should include a target Peer identifier
Signed-off-by: Peter Somogyi <psomogyi@apache.org>
1 parent eef8e14 commit 3b0013d

File tree

2 files changed

+78
-44
lines changed

2 files changed

+78
-44
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java

Lines changed: 35 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -198,11 +198,14 @@ private void connectToPeers() {
198198
protected boolean sleepForRetries(String msg, int sleepMultiplier) {
199199
try {
200200
if (LOG.isTraceEnabled()) {
201-
LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
201+
LOG.trace("{} {}, sleeping {} times {}",
202+
logPeerId(), msg, sleepForRetries, sleepMultiplier);
202203
}
203204
Thread.sleep(this.sleepForRetries * sleepMultiplier);
204205
} catch (InterruptedException e) {
205-
LOG.debug("Interrupted while sleeping between retries");
206+
if (LOG.isDebugEnabled()) {
207+
LOG.debug("{} Interrupted while sleeping between retries", logPeerId());
208+
}
206209
}
207210
return sleepMultiplier < maxRetriesMultiplier;
208211
}
@@ -288,7 +291,7 @@ private void reconnectToPeerCluster() {
288291
connection =
289292
ClusterConnectionFactory.createAsyncClusterConnection(conf, null, User.getCurrent());
290293
} catch (IOException ioe) {
291-
LOG.warn("Failed to create connection for peer cluster", ioe);
294+
LOG.warn("{} Failed to create connection for peer cluster", logPeerId(), ioe);
292295
}
293296
if (connection != null) {
294297
this.conn = connection;
@@ -301,8 +304,10 @@ private long parallelReplicate(CompletionService<Integer> pool, ReplicateContext
301304
for (int i = 0; i < batches.size(); i++) {
302305
List<Entry> entries = batches.get(i);
303306
if (!entries.isEmpty()) {
304-
LOG.trace("Submitting {} entries of total size {}", entries.size(),
305-
replicateContext.getSize());
307+
if (LOG.isTraceEnabled()) {
308+
LOG.trace("{} Submitting {} entries of total size {}", logPeerId(), entries.size(),
309+
replicateContext.getSize());
310+
}
306311
// RuntimeExceptions encountered here bubble up and are handled in ReplicationSource
307312
pool.submit(createReplicator(entries, i));
308313
futures++;
@@ -353,8 +358,8 @@ public boolean replicate(ReplicateContext replicateContext) {
353358

354359
int numSinks = replicationSinkMgr.getNumSinks();
355360
if (numSinks == 0) {
356-
LOG.warn("No replication sinks found, returning without replicating. The source should " +
357-
"retry with the same set of edits.");
361+
LOG.warn("{} No replication sinks found, returning without replicating. "
362+
+ "The source should retry with the same set of edits.", logPeerId());
358363
return false;
359364
}
360365

@@ -376,7 +381,8 @@ public boolean replicate(ReplicateContext replicateContext) {
376381
} catch (IOException ioe) {
377382
if (ioe instanceof RemoteException) {
378383
ioe = ((RemoteException) ioe).unwrapRemoteException();
379-
LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe);
384+
LOG.warn("{} Can't replicate because of an error on the remote cluster: ", logPeerId(),
385+
ioe);
380386
if (ioe instanceof TableNotFoundException) {
381387
if (dropOnDeletedTables) {
382388
// this is a bit fragile, but cannot change how TNFE is serialized
@@ -389,19 +395,20 @@ public boolean replicate(ReplicateContext replicateContext) {
389395
// Would potentially be better to retry in one of the outer loops
390396
// and add a table filter there; but that would break the encapsulation,
391397
// so we're doing the filtering here.
392-
LOG.info("Missing table detected at sink, local table also does not exist, " +
393-
"filtering edits for '" + table + "'");
398+
LOG.info("{} Missing table detected at sink, local table also does not "
399+
+ "exist, filtering edits for '{}'", logPeerId(), table);
394400
batches = filterBatches(batches, table);
395401
continue;
396402
}
397403
} catch (IOException iox) {
398-
LOG.warn("Exception checking for local table: ", iox);
404+
LOG.warn("{} Exception checking for local table: ", logPeerId(), iox);
399405
}
400406
}
401407
}
402408
// fall through and sleep below
403409
} else {
404-
LOG.warn("Peer encountered RemoteException, rechecking all sinks: ", ioe);
410+
LOG.warn("{} Peer encountered RemoteException, rechecking all sinks: ", logPeerId(),
411+
ioe);
405412
replicationSinkMgr.chooseSinks();
406413
}
407414
} else {
@@ -414,10 +421,10 @@ public boolean replicate(ReplicateContext replicateContext) {
414421
"caused by a machine failure or a massive slowdown",
415422
this.socketTimeoutMultiplier);
416423
} else if (ioe instanceof ConnectException || ioe instanceof UnknownHostException) {
417-
LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
424+
LOG.warn("{} Peer is unavailable, rechecking all sinks: ", logPeerId(), ioe);
418425
replicationSinkMgr.chooseSinks();
419426
} else {
420-
LOG.warn("Can't replicate because of a local or network error: ", ioe);
427+
LOG.warn("{} Can't replicate because of a local or network error: ", logPeerId(), ioe);
421428
}
422429
}
423430
if (sleepForRetries("Since we are unable to replicate", sleepMultiplier)) {
@@ -440,7 +447,7 @@ protected void doStop() {
440447
this.conn.close();
441448
this.conn = null;
442449
} catch (IOException e) {
443-
LOG.warn("Failed to close the connection");
450+
LOG.warn("{} Failed to close the connection", logPeerId());
444451
}
445452
}
446453
// Allow currently running replication tasks to finish
@@ -466,18 +473,22 @@ protected int replicateEntries(List<Entry> entries, int batchIndex) throws IOExc
466473
int entriesHashCode = System.identityHashCode(entries);
467474
if (LOG.isTraceEnabled()) {
468475
long size = entries.stream().mapToLong(this::getEstimatedEntrySize).sum();
469-
LOG.trace("Replicating batch {} of {} entries with total size {} bytes to {}",
470-
entriesHashCode, entries.size(), size, replicationClusterId);
476+
LOG.trace("{} Replicating batch {} of {} entries with total size {} bytes to {}",
477+
logPeerId(), entriesHashCode, entries.size(), size, replicationClusterId);
471478
}
472479
sinkPeer = replicationSinkMgr.getReplicationSink();
473480
AsyncRegionServerAdmin rsAdmin = sinkPeer.getRegionServer();
474481
try {
475482
ReplicationProtbufUtil.replicateWALEntry(rsAdmin,
476483
entries.toArray(new Entry[entries.size()]), replicationClusterId, baseNamespaceDir,
477484
hfileArchiveDir);
478-
LOG.trace("Completed replicating batch {}", entriesHashCode);
485+
if (LOG.isTraceEnabled()) {
486+
LOG.trace("{} Completed replicating batch {}", logPeerId(), entriesHashCode);
487+
}
479488
} catch (IOException e) {
480-
LOG.trace("Failed replicating batch {}", entriesHashCode, e);
489+
if (LOG.isTraceEnabled()) {
490+
LOG.trace("{} Failed replicating batch {}", logPeerId(), entriesHashCode, e);
491+
}
481492
throw e;
482493
}
483494
replicationSinkMgr.reportSinkSuccess(sinkPeer);
@@ -515,4 +526,9 @@ protected Callable<Integer> createReplicator(List<Entry> entries, int batchIndex
515526
return isSerial ? () -> serialReplicateRegionEntries(entries, batchIndex)
516527
: () -> replicateEntries(entries, batchIndex);
517528
}
529+
530+
private String logPeerId(){
531+
return "[Source for peer " + this.ctx.getPeerId() + "]:";
532+
}
533+
518534
}

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java

Lines changed: 43 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -209,14 +209,17 @@ public void enqueueLog(Path log) {
209209
} else {
210210
queue.put(log);
211211
}
212-
LOG.trace("Added log file {} to queue of source {}.", logPrefix,
212+
if (LOG.isTraceEnabled()) {
213+
LOG.trace("{} Added log file {} to queue of source {}.", logPeerId(), logPrefix,
213214
this.replicationQueueInfo.getQueueId());
215+
}
214216
this.metrics.incrSizeOfLogQueue();
215217
// This will log a warning for each new log that gets created above the warn threshold
216218
int queueSize = queue.size();
217219
if (queueSize > this.logQueueWarnThreshold) {
218-
LOG.warn("WAL group " + logPrefix + " queue size: " + queueSize
219-
+ " exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
220+
LOG.warn("{} WAL group {} queue size: {} exceeds value of "
221+
+ "replication.source.log.queue.warn: {}", logPeerId(),
222+
logPrefix, queueSize, logQueueWarnThreshold);
220223
}
221224
}
222225

@@ -232,8 +235,8 @@ public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Pat
232235
this.queueStorage.addHFileRefs(peerId, pairs);
233236
metrics.incrSizeOfHFileRefsQueue(pairs.size());
234237
} else {
235-
LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
236-
+ Bytes.toString(family) + " to peer id " + peerId);
238+
LOG.debug("HFiles will not be replicated belonging to the table {} family {} to peer id {}",
239+
tableName, Bytes.toString(family), peerId);
237240
}
238241
} else {
239242
// user has explicitly not defined any table cfs for replication, means replicate all the
@@ -305,9 +308,14 @@ private void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> q
305308
ReplicationSourceShipper worker = createNewShipper(walGroupId, queue);
306309
ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker);
307310
if (extant != null) {
308-
LOG.debug("Someone has beat us to start a worker thread for wal group {}", walGroupId);
311+
if(LOG.isDebugEnabled()) {
312+
LOG.debug("{} Someone has beat us to start a worker thread for wal group {}", logPeerId(),
313+
walGroupId);
314+
}
309315
} else {
310-
LOG.debug("Starting up worker for wal group {}", walGroupId);
316+
if(LOG.isDebugEnabled()) {
317+
LOG.debug("{} Starting up worker for wal group {}", logPeerId(), walGroupId);
318+
}
311319
ReplicationSourceWALReader walReader =
312320
createNewWALReader(walGroupId, queue, worker.getStartPosition());
313321
Threads.setDaemonThreadRunning(walReader, Thread.currentThread().getName() +
@@ -337,7 +345,7 @@ public Map<String, ReplicationStatus> getWalGroupStatus() {
337345
}
338346
} else {
339347
currentPath = new Path("NO_LOGS_IN_QUEUE");
340-
LOG.warn("No replication ongoing, waiting for new log");
348+
LOG.warn("{} No replication ongoing, waiting for new log", logPeerId());
341349
}
342350
ReplicationStatus.ReplicationStatusBuilder statusBuilder = ReplicationStatus.newBuilder();
343351
statusBuilder.withPeerId(this.getPeerId())
@@ -378,7 +386,8 @@ private ReplicationSourceWALReader createNewWALReader(String walGroupId,
378386

379387
protected final void uncaughtException(Thread t, Throwable e) {
380388
RSRpcServices.exitIfOOME(e);
381-
LOG.error("Unexpected exception in " + t.getName() + " currentPath=" + getCurrentPath(), e);
389+
LOG.error("Unexpected exception in {} currentPath={}",
390+
t.getName(), getCurrentPath(), e);
382391
server.abort("Unexpected exception in " + t.getName(), e);
383392
}
384393

@@ -399,7 +408,7 @@ public void tryThrottle(int batchSize) throws InterruptedException {
399408
long sleepTicks = throttler.getNextSleepInterval(batchSize);
400409
if (sleepTicks > 0) {
401410
if (LOG.isTraceEnabled()) {
402-
LOG.trace("To sleep " + sleepTicks + "ms for throttling control");
411+
LOG.trace("{} To sleep {}ms for throttling control", logPeerId(), sleepTicks);
403412
}
404413
Thread.sleep(sleepTicks);
405414
// reset throttler's cycle start tick when sleep for throttling occurs
@@ -433,11 +442,14 @@ private long getCurrentBandwidth() {
433442
protected boolean sleepForRetries(String msg, int sleepMultiplier) {
434443
try {
435444
if (LOG.isTraceEnabled()) {
436-
LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
445+
LOG.trace("{} {}, sleeping {} times {}",
446+
logPeerId(), msg, sleepForRetries, sleepMultiplier);
437447
}
438448
Thread.sleep(this.sleepForRetries * sleepMultiplier);
439449
} catch (InterruptedException e) {
440-
LOG.debug("Interrupted while sleeping between retries");
450+
if(LOG.isDebugEnabled()) {
451+
LOG.debug("{} Interrupted while sleeping between retries", logPeerId());
452+
}
441453
Thread.currentThread().interrupt();
442454
}
443455
return sleepMultiplier < maxRetriesMultiplier;
@@ -450,7 +462,7 @@ private void initialize() {
450462
try {
451463
replicationEndpoint = createReplicationEndpoint();
452464
} catch (Exception e) {
453-
LOG.warn("error creating ReplicationEndpoint, retry", e);
465+
LOG.warn("{} error creating ReplicationEndpoint, retry", logPeerId(), e);
454466
if (sleepForRetries("Error creating ReplicationEndpoint", sleepMultiplier)) {
455467
sleepMultiplier++;
456468
}
@@ -462,7 +474,7 @@ private void initialize() {
462474
this.replicationEndpoint = replicationEndpoint;
463475
break;
464476
} catch (Exception e) {
465-
LOG.warn("Error starting ReplicationEndpoint, retry", e);
477+
LOG.warn("{} Error starting ReplicationEndpoint, retry", logPeerId(), e);
466478
replicationEndpoint.stop();
467479
if (sleepForRetries("Error starting ReplicationEndpoint", sleepMultiplier)) {
468480
sleepMultiplier++;
@@ -480,8 +492,10 @@ private void initialize() {
480492
for (;;) {
481493
peerClusterId = replicationEndpoint.getPeerUUID();
482494
if (this.isSourceActive() && peerClusterId == null) {
483-
LOG.debug("Could not connect to Peer ZK. Sleeping for "
484-
+ (this.sleepForRetries * sleepMultiplier) + " millis.");
495+
if(LOG.isDebugEnabled()) {
496+
LOG.debug("{} Could not connect to Peer ZK. Sleeping for {} millis", logPeerId(),
497+
(this.sleepForRetries * sleepMultiplier));
498+
}
485499
if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
486500
sleepMultiplier++;
487501
}
@@ -499,8 +513,8 @@ private void initialize() {
499513
this.manager.removeSource(this);
500514
return;
501515
}
502-
LOG.info("Source: {}, is now replicating from cluster: {}; to peer cluster: {};",
503-
this.replicationQueueInfo.getQueueId(), clusterId, peerClusterId);
516+
LOG.info("{} Source: {}, is now replicating from cluster: {}; to peer cluster: {};",
517+
logPeerId(), this.replicationQueueInfo.getQueueId(), clusterId, peerClusterId);
504518

505519
initializeWALEntryFilter(peerClusterId);
506520
// start workers
@@ -533,10 +547,10 @@ public void terminate(String reason, Exception cause) {
533547

534548
public void terminate(String reason, Exception cause, boolean join) {
535549
if (cause == null) {
536-
LOG.info("Closing source " + this.queueId + " because: " + reason);
550+
LOG.info("{} Closing source {} because: {}", logPeerId(), this.queueId, reason);
537551
} else {
538-
LOG.error("Closing source " + this.queueId + " because an error occurred: " + reason,
539-
cause);
552+
LOG.error("{} Closing source {} because an error occurred: {}",
553+
logPeerId(), this.queueId, reason, cause);
540554
}
541555
this.sourceRunning = false;
542556
if (initThread != null && Thread.currentThread() != initThread) {
@@ -560,7 +574,7 @@ public void terminate(String reason, Exception cause, boolean join) {
560574
// Wait worker to stop
561575
Thread.sleep(this.sleepForRetries);
562576
} catch (InterruptedException e) {
563-
LOG.info("Interrupted while waiting " + worker.getName() + " to stop");
577+
LOG.info("{} Interrupted while waiting {} to stop", logPeerId(), worker.getName());
564578
Thread.currentThread().interrupt();
565579
}
566580
// If worker still is alive after waiting, interrupt it
@@ -581,15 +595,15 @@ public void terminate(String reason, Exception cause, boolean join) {
581595
if (join) {
582596
for (ReplicationSourceShipper worker : workers) {
583597
Threads.shutdown(worker, this.sleepForRetries);
584-
LOG.info("ReplicationSourceWorker " + worker.getName() + " terminated");
598+
LOG.info("{} ReplicationSourceWorker {} terminated", logPeerId(), worker.getName());
585599
}
586600
if (this.replicationEndpoint != null) {
587601
try {
588602
this.replicationEndpoint.awaitTerminated(sleepForRetries * maxRetriesMultiplier,
589603
TimeUnit.MILLISECONDS);
590604
} catch (TimeoutException te) {
591-
LOG.warn("Got exception while waiting for endpoint to shutdown for replication source :" +
592-
this.queueId, te);
605+
LOG.warn("{} Got exception while waiting for endpoint to shutdown "
606+
+ "for replication source : {}", logPeerId(), this.queueId, te);
593607
}
594608
}
595609
}
@@ -721,4 +735,8 @@ ReplicationQueueStorage getQueueStorage() {
721735
void removeWorker(ReplicationSourceShipper worker) {
722736
workerThreads.remove(worker.walGroupId, worker);
723737
}
738+
739+
private String logPeerId(){
740+
return "[Source for peer " + this.getPeer().getId() + "]:";
741+
}
724742
}

0 commit comments

Comments
 (0)