You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/01/15 07:03:14 UTC
svn commit: r1558301 -
/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
Author: liangxie
Date: Wed Jan 15 06:03:14 2014
New Revision: 1558301
URL: http://svn.apache.org/r1558301
Log:
HBASE-10335 AuthFailedException in zookeeper may block replication forever (Liu Shaohui)
Modified:
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
Modified: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java?rev=1558301&r1=1558300&r2=1558301&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java (original)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java Wed Jan 15 06:03:14 2014
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.zookeeper
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.AuthFailedException;
import org.apache.zookeeper.KeeperException.ConnectionLossException;
import org.apache.zookeeper.KeeperException.SessionExpiredException;
@@ -226,6 +227,9 @@ public class ReplicationPeersZKImpl exte
try {
addresses = fetchSlavesAddresses(peer.getZkw());
} catch (KeeperException ke) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Fetch salves addresses failed.", ke);
+ }
reconnectPeer(ke, peer);
addresses = Collections.emptyList();
}
@@ -332,7 +336,8 @@ public class ReplicationPeersZKImpl exte
* @param peer
*/
private void reconnectPeer(KeeperException ke, ReplicationPeer peer) {
- if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException) {
+ if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException
+ || ke instanceof AuthFailedException) {
LOG.warn("Lost the ZooKeeper connection for peer " + peer.getClusterKey(), ke);
try {
peer.reloadZkWatcher();
@@ -491,4 +496,4 @@ public class ReplicationPeersZKImpl exte
}
}
-}
\ No newline at end of file
+}