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