From a98e6fec52bea7a77fabc87cb3741bf36048c545 Mon Sep 17 00:00:00 2001 From: Xu Cang Date: Wed, 7 Aug 2019 17:10:48 -0700 Subject: [PATCH] HBASE-22775 Enhance logging for peer related operations --- .../replication/ZKReplicationPeerStorage.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java index 7a943c4035..5c39cd7feb 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java @@ -31,8 +31,10 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; /** @@ -42,6 +44,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; public class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements ReplicationPeerStorage { + private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationPeerStorage.class); public static final String PEERS_ZNODE = "zookeeper.znode.replication.peers"; public static final String PEERS_ZNODE_DEFAULT = "peers"; @@ -101,6 +104,8 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase @Override public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled, SyncReplicationState syncReplicationState) throws ReplicationException { + LOG.info("addPeer: peerId:["+ peerId + "], enabled:" + enabled + ", ReplicationPeerConfg:" + + peerConfig + ", syncReplicationState: " + syncReplicationState.value()); List multiOps = Arrays.asList( ZKUtilOp.createAndFailSilent(getPeerNode(peerId), ReplicationPeerConfigUtil.toByteArray(peerConfig)), @@ -122,8 +127,10 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase @Override public void removePeer(String peerId) throws ReplicationException { + try { ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId)); + LOG.info("removePeer: peerId:[ "+ peerId + "]"); } catch (KeeperException e) { throw new ReplicationException("Could not remove peer with id=" + peerId, e); } @@ -134,6 +141,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase byte[] stateBytes = enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES; try { ZKUtil.setData(zookeeper, getPeerStateNode(peerId), stateBytes); + LOG.info("setPeerState: peerId:["+ peerId + "], enabled:" + enabled); } catch (KeeperException e) { throw new ReplicationException("Unable to change state of the peer with id=" + peerId, e); } @@ -145,6 +153,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase try { ZKUtil.setData(this.zookeeper, getPeerNode(peerId), ReplicationPeerConfigUtil.toByteArray(peerConfig)); + LOG.info("updatePeerConfig: peerId:["+ peerId + "], replicationPeerConfig:" + peerConfig); } catch (KeeperException e) { throw new ReplicationException( "There was a problem trying to save changes to the " + "replication peer " + peerId, e); @@ -197,6 +206,8 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase try { ZKUtil.createSetData(zookeeper, getNewSyncReplicationStateNode(peerId), SyncReplicationState.toByteArray(state)); + LOG.info("setPeerNewSyncReplicationState: peerId:[ "+ peerId + "], SyncReplicationState:" + + state.value()); } catch (KeeperException e) { throw new ReplicationException( "Unable to set the new sync replication state for peer with id=" + peerId, e); @@ -212,6 +223,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase Arrays.asList(ZKUtilOp.setData(newStateNode, NONE_STATE_ZNODE_BYTES), ZKUtilOp.setData(getSyncReplicationStateNode(peerId), data)), false); + LOG.info("transitPeerSyncReplicationState: peerId:[ "+ peerId + "]"); } catch (KeeperException | InterruptedException e) { throw new ReplicationException( "Error transiting sync replication state for peer with id=" + peerId, e); -- 2.17.2 (Apple Git-113)