001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.zookeeper; 019 020import java.io.BufferedReader; 021import java.io.BufferedWriter; 022import java.io.IOException; 023import java.io.InputStreamReader; 024import java.io.OutputStreamWriter; 025import java.io.PrintWriter; 026import java.net.InetSocketAddress; 027import java.net.Socket; 028import java.net.UnknownHostException; 029import java.nio.charset.StandardCharsets; 030import java.util.ArrayList; 031import java.util.Collections; 032import java.util.LinkedList; 033import java.util.List; 034import org.apache.hadoop.hbase.HConstants; 035import org.apache.hadoop.hbase.TableName; 036import org.apache.hadoop.hbase.exceptions.DeserializationException; 037import org.apache.yetus.audience.InterfaceAudience; 038import org.apache.zookeeper.KeeperException; 039import org.slf4j.Logger; 040import org.slf4j.LoggerFactory; 041 042import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 043 044import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 045import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; 046 047/** 048 * Builds a string containing everything in ZooKeeper. This is inherently invasive into the 049 * structures of other components' logical responsibilities. 050 */ 051@InterfaceAudience.Private 052public final class ZKDump { 053 private static final Logger LOG = LoggerFactory.getLogger(ZKDump.class); 054 055 private ZKDump() { 056 } 057 058 public static String dump(final ZKWatcher zkWatcher) { 059 final int zkDumpConnectionTimeOut = 060 zkWatcher.getConfiguration().getInt("zookeeper.dump.connection.timeout", 1000); 061 StringBuilder sb = new StringBuilder(); 062 try { 063 sb.append("HBase is rooted at ").append(zkWatcher.getZNodePaths().baseZNode); 064 sb.append("\nActive master address: "); 065 try { 066 sb.append("\n ").append(MasterAddressTracker.getMasterAddress(zkWatcher)); 067 } catch (IOException e) { 068 sb.append("<<FAILED LOOKUP: ").append(e.getMessage()).append(">>"); 069 } 070 sb.append("\nBackup master addresses:"); 071 final List<String> backupMasterChildrenNoWatchList = 072 ZKUtil.listChildrenNoWatch(zkWatcher, zkWatcher.getZNodePaths().backupMasterAddressesZNode); 073 if (backupMasterChildrenNoWatchList != null) { 074 for (String child : backupMasterChildrenNoWatchList) { 075 sb.append("\n ").append(child); 076 } 077 } 078 sb.append("\nRegion server holding ").append(TableName.META_TABLE_NAME).append(":"); 079 sb.append("\n ").append(MetaTableLocator.getMetaRegionLocation(zkWatcher)); 080 int numMetaReplicas = zkWatcher.getMetaReplicaNodes().size(); 081 for (int i = 1; i < numMetaReplicas; i++) { 082 sb.append("\n").append(" replica").append(i).append(": ") 083 .append(MetaTableLocator.getMetaRegionLocation(zkWatcher, i)); 084 } 085 sb.append("\nRegion servers:"); 086 final List<String> rsChildrenNoWatchList = 087 ZKUtil.listChildrenNoWatch(zkWatcher, zkWatcher.getZNodePaths().rsZNode); 088 if (rsChildrenNoWatchList != null) { 089 for (String child : rsChildrenNoWatchList) { 090 sb.append("\n ").append(child); 091 } 092 } 093 try { 094 getReplicationZnodesDump(zkWatcher, sb); 095 } catch (KeeperException ke) { 096 LOG.warn("Couldn't get the replication znode dump", ke); 097 } 098 sb.append("\nQuorum Server Statistics:"); 099 String[] servers = zkWatcher.getQuorum().split(","); 100 for (String server : servers) { 101 sb.append("\n ").append(server); 102 try { 103 String[] stat = getServerStats(server, zkDumpConnectionTimeOut); 104 105 if (stat == null) { 106 sb.append("[Error] invalid quorum server: ").append(server); 107 break; 108 } 109 110 for (String s : stat) { 111 sb.append("\n ").append(s); 112 } 113 } catch (Exception e) { 114 sb.append("\n ERROR: ").append(e.getMessage()); 115 } 116 } 117 } catch (KeeperException ke) { 118 sb.append("\nFATAL ZooKeeper Exception!\n"); 119 sb.append("\n").append(ke.getMessage()); 120 } 121 return sb.toString(); 122 } 123 124 /** 125 * Appends replication znodes to the passed StringBuilder. 126 * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation 127 * @param sb the {@link StringBuilder} to append to 128 * @throws KeeperException if a ZooKeeper operation fails 129 */ 130 private static void getReplicationZnodesDump(ZKWatcher zkw, StringBuilder sb) 131 throws KeeperException { 132 String replicationZnode = zkw.getZNodePaths().replicationZNode; 133 134 if (ZKUtil.checkExists(zkw, replicationZnode) == -1) { 135 return; 136 } 137 138 // do a ls -r on this znode 139 sb.append("\n").append(replicationZnode).append(": "); 140 List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode); 141 if (children != null) { 142 Collections.sort(children); 143 for (String child : children) { 144 String zNode = ZNodePaths.joinZNode(replicationZnode, child); 145 if (zNode.equals(zkw.getZNodePaths().peersZNode)) { 146 appendPeersZnodes(zkw, zNode, sb); 147 } else if (zNode.equals(zkw.getZNodePaths().queuesZNode)) { 148 appendRSZnodes(zkw, zNode, sb); 149 } else if (zNode.equals(zkw.getZNodePaths().hfileRefsZNode)) { 150 appendHFileRefsZNodes(zkw, zNode, sb); 151 } 152 } 153 } 154 } 155 156 private static void appendHFileRefsZNodes(ZKWatcher zkw, String hFileRefsZNode, StringBuilder sb) 157 throws KeeperException { 158 sb.append("\n").append(hFileRefsZNode).append(": "); 159 final List<String> hFileRefChildrenNoWatchList = 160 ZKUtil.listChildrenNoWatch(zkw, hFileRefsZNode); 161 if (hFileRefChildrenNoWatchList != null) { 162 for (String peerIdZNode : hFileRefChildrenNoWatchList) { 163 String zNodeToProcess = ZNodePaths.joinZNode(hFileRefsZNode, peerIdZNode); 164 sb.append("\n").append(zNodeToProcess).append(": "); 165 List<String> peerHFileRefsZNodes = ZKUtil.listChildrenNoWatch(zkw, zNodeToProcess); 166 if (peerHFileRefsZNodes != null) { 167 sb.append(String.join(", ", peerHFileRefsZNodes)); 168 } 169 } 170 } 171 } 172 173 /** 174 * Returns a string with replication znodes and position of the replication log 175 * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation 176 * @return aq string of replication znodes and log positions 177 */ 178 public static String getReplicationZnodesDump(ZKWatcher zkw) throws KeeperException { 179 StringBuilder sb = new StringBuilder(); 180 getReplicationZnodesDump(zkw, sb); 181 return sb.toString(); 182 } 183 184 private static void appendRSZnodes(ZKWatcher zkw, String znode, StringBuilder sb) 185 throws KeeperException { 186 List<String> stack = new LinkedList<>(); 187 stack.add(znode); 188 do { 189 String znodeToProcess = stack.remove(stack.size() - 1); 190 sb.append("\n").append(znodeToProcess).append(": "); 191 byte[] data; 192 try { 193 data = ZKUtil.getData(zkw, znodeToProcess); 194 } catch (InterruptedException e) { 195 zkw.interruptedException(e); 196 return; 197 } 198 if (data != null && data.length > 0) { // log position 199 long position = 0; 200 try { 201 position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess)); 202 sb.append(position); 203 } catch (DeserializationException ignored) { 204 } catch (InterruptedException e) { 205 zkw.interruptedException(e); 206 return; 207 } 208 } 209 for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) { 210 stack.add(ZNodePaths.joinZNode(znodeToProcess, zNodeChild)); 211 } 212 } while (stack.size() > 0); 213 } 214 215 private static void appendPeersZnodes(ZKWatcher zkw, String peersZnode, StringBuilder sb) 216 throws KeeperException { 217 int pblen = ProtobufUtil.lengthOfPBMagic(); 218 sb.append("\n").append(peersZnode).append(": "); 219 for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) { 220 String znodeToProcess = ZNodePaths.joinZNode(peersZnode, peerIdZnode); 221 byte[] data; 222 try { 223 data = ZKUtil.getData(zkw, znodeToProcess); 224 } catch (InterruptedException e) { 225 zkw.interruptedException(e); 226 return; 227 } 228 // parse the data of the above peer znode. 229 try { 230 ReplicationProtos.ReplicationPeer.Builder builder = 231 ReplicationProtos.ReplicationPeer.newBuilder(); 232 ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen); 233 String clusterKey = builder.getClusterkey(); 234 sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey); 235 // add the peer-state. 236 appendPeerState(zkw, znodeToProcess, sb); 237 } catch (IOException ipbe) { 238 LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe); 239 } 240 } 241 } 242 243 private static void appendPeerState(ZKWatcher zkw, String znodeToProcess, StringBuilder sb) 244 throws KeeperException, InvalidProtocolBufferException { 245 String peerState = 246 zkw.getConfiguration().get("zookeeper.znode.replication.peers.state", "peer-state"); 247 int pblen = ProtobufUtil.lengthOfPBMagic(); 248 for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) { 249 if (!child.equals(peerState)) { 250 continue; 251 } 252 253 String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child); 254 sb.append("\n").append(peerStateZnode).append(": "); 255 byte[] peerStateData; 256 try { 257 peerStateData = ZKUtil.getData(zkw, peerStateZnode); 258 ReplicationProtos.ReplicationState.Builder builder = 259 ReplicationProtos.ReplicationState.newBuilder(); 260 ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen); 261 sb.append(builder.getState().name()); 262 } catch (IOException ipbe) { 263 LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe); 264 } catch (InterruptedException e) { 265 zkw.interruptedException(e); 266 return; 267 } 268 } 269 } 270 271 /** 272 * Gets the statistics from the given server. 273 * @param server The server to get the statistics from. 274 * @param timeout The socket timeout to use. 275 * @return The array of response strings. 276 * @throws IOException When the socket communication fails. 277 */ 278 private static String[] getServerStats(String server, int timeout) throws IOException { 279 String[] sp = server.split(":"); 280 if (sp.length == 0) { 281 return null; 282 } 283 284 String host = sp[0]; 285 int port = sp.length > 1 ? Integer.parseInt(sp[1]) : HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT; 286 287 try (Socket socket = new Socket()) { 288 InetSocketAddress sockAddr = new InetSocketAddress(host, port); 289 if (sockAddr.isUnresolved()) { 290 throw new UnknownHostException(host + " cannot be resolved"); 291 } 292 socket.connect(sockAddr, timeout); 293 socket.setSoTimeout(timeout); 294 try ( 295 PrintWriter out = new PrintWriter(new BufferedWriter( 296 new OutputStreamWriter(socket.getOutputStream(), StandardCharsets.UTF_8)), true); 297 BufferedReader in = new BufferedReader( 298 new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8))) { 299 out.println("stat"); 300 out.flush(); 301 ArrayList<String> res = new ArrayList<>(); 302 while (true) { 303 String line = in.readLine(); 304 if (line != null) { 305 res.add(line); 306 } else { 307 break; 308 } 309 } 310 return res.toArray(new String[res.size()]); 311 } 312 } 313 } 314}