hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r1520184 - /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
Date Thu, 05 Sep 2013 00:44:57 GMT
Author: stack
Date: Thu Sep  5 00:44:57 2013
New Revision: 1520184

URL: http://svn.apache.org/r1520184
Log:
HBASE-9388 [replication] ZK Dump prints the raw PBUF for the HLog positions

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1520184&r1=1520183&r2=1520184&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Thu
Sep  5 00:44:57 2013
@@ -1623,6 +1623,12 @@ public class ZKUtil {
     return sb.toString();
   }
 
+  /**
+   * Appends replication znodes to the passed StringBuilder.
+   * @param zkw
+   * @param sb
+   * @throws KeeperException
+   */
   private static void getReplicationZnodesDump(ZooKeeperWatcher zkw, StringBuilder sb)
       throws KeeperException {
     String replicationZNodeName = zkw.getConfiguration().get("zookeeper.znode.replication",
@@ -1630,18 +1636,76 @@ public class ZKUtil {
     String replicationZnode = joinZNode(zkw.baseZNode, replicationZNodeName);
     if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
     // do a ls -r on this znode
+    sb.append("\n").append(replicationZnode).append(": ");
+    List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
+    for (String child : children) {
+      String znode = joinZNode(replicationZnode, child);
+      if (child.equals(zkw.getConfiguration().get("zookeeper.znode.replication.peers", "peers")))
{
+        appendPeersZnodes(zkw, znode, sb);
+      } else if (child.equals(zkw.getConfiguration().
+          get("zookeeper.znode.replication.rs", "rs"))) {
+        appendRSZnodes(zkw, znode, sb);
+      }
+    }
+  }
+
+  private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb)
+      throws KeeperException {
     List<String> stack = new LinkedList<String>();
-    stack.add(replicationZnode);
+    stack.add(znode);
     do {
       String znodeToProcess = stack.remove(stack.size() - 1);
-      sb.append("\n").append(znodeToProcess).append(": ")
-          .append(Bytes.toString(ZKUtil.getData(zkw, znodeToProcess)));
+      sb.append("\n").append(znodeToProcess).append(": ");
+      byte[] data = ZKUtil.getData(zkw, znodeToProcess);
+      if (data != null && data.length > 0) { // log position
+        long position = 0;
+        try {
+          position = ZKUtil.parseHLogPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
+          sb.append(position);
+        } catch (Exception e) {
+        }
+      }
       for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
         stack.add(ZKUtil.joinZNode(znodeToProcess, zNodeChild));
       }
     } while (stack.size() > 0);
   }
 
+  private static void appendPeersZnodes(ZooKeeperWatcher zkw, String peersZnode,
+    StringBuilder sb) throws KeeperException {
+    int pblen = ProtobufUtil.lengthOfPBMagic();
+    sb.append("\n").append(peersZnode).append(": ");
+    for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
+      String znodeToProcess = ZKUtil.joinZNode(peersZnode, peerIdZnode);
+      byte[] data = ZKUtil.getData(zkw, znodeToProcess);
+      // parse the data of the above peer znode.
+      try {
+      String clusterKey = ZooKeeperProtos.ReplicationPeer.newBuilder().
+        mergeFrom(data, pblen, data.length - pblen).getClusterkey();
+      sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
+      // add the peer-state.
+      appendPeerState(zkw, znodeToProcess, sb);
+      } catch (InvalidProtocolBufferException ipbe) {
+        LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
+      }
+    }
+  }
+
+  private static void appendPeerState(ZooKeeperWatcher zkw, String znodeToProcess,
+      StringBuilder sb) throws KeeperException, InvalidProtocolBufferException {
+    String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
+      "peer-state");
+    int pblen = ProtobufUtil.lengthOfPBMagic();
+    for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
+      if (!child.equals(peerState)) continue;
+      String peerStateZnode = ZKUtil.joinZNode(znodeToProcess, child);
+      sb.append("\n").append(peerStateZnode).append(": ");
+      byte[] peerStateData = ZKUtil.getData(zkw, peerStateZnode);
+      sb.append(ZooKeeperProtos.ReplicationState.newBuilder()
+          .mergeFrom(peerStateData, pblen, peerStateData.length - pblen).getState().name());
+    }
+  }
+
   /**
    * Gets the statistics from the given server.
    *



Mime
View raw message