You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2020/09/16 10:01:06 UTC
[hbase] branch branch-2 updated: HBASE-25012 HBASE-24359 causes
replication missed log of some RemoteException (#2383)
This is an automated email from the ASF dual-hosted git repository.
zghao pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2 by this push:
new d150749 HBASE-25012 HBASE-24359 causes replication missed log of some RemoteException (#2383)
d150749 is described below
commit d15074981f8925676188b5a5c2e68cc9f989785d
Author: XinSun <dd...@gmail.com>
AuthorDate: Wed Sep 16 18:00:35 2020 +0800
HBASE-25012 HBASE-24359 causes replication missed log of some RemoteException (#2383)
Signed-off-by: Wellington Chevreuil <wc...@apache.org>
Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
.../regionserver/HBaseInterClusterReplicationEndpoint.java | 8 ++++++--
1 file changed, 6 insertions(+), 2 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 4fb6146..7287c61 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -559,10 +559,14 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
} else if (dropOnDeletedColumnFamilies && isNoSuchColumnFamilyException(ioe)) {
batches = filterNotExistColumnFamilyEdits(batches);
if (batches.isEmpty()) {
- LOG.warn(
- "After filter not exist column family's edits, 0 edits to replicate, just return");
+ LOG.warn("After filter not exist column family's edits, 0 edits to replicate, "
+ + "just return");
return true;
}
+ } else {
+ LOG.warn("{} Peer encountered RemoteException, rechecking all sinks: ", logPeerId(),
+ ioe);
+ replicationSinkMgr.chooseSinks();
}
} else {
if (ioe instanceof SocketTimeoutException) {