1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.replication.regionserver;
19
20 import org.apache.commons.logging.Log;
21 import org.apache.commons.logging.LogFactory;
22 import org.apache.hadoop.conf.Configuration;
23 import org.apache.hadoop.conf.Configured;
24 import org.apache.hadoop.fs.FileStatus;
25 import org.apache.hadoop.fs.FileSystem;
26 import org.apache.hadoop.fs.Path;
27 import org.apache.hadoop.hbase.*;
28 import org.apache.hadoop.hbase.client.ClusterConnection;
29 import org.apache.hadoop.hbase.client.ConnectionFactory;
30 import org.apache.hadoop.hbase.client.HBaseAdmin;
31 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
32 import org.apache.hadoop.hbase.io.WALLink;
33 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
34 import org.apache.hadoop.hbase.replication.*;
35 import org.apache.hadoop.hbase.util.FSUtils;
36 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
37 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
38 import org.apache.hadoop.util.Tool;
39 import org.apache.hadoop.util.ToolRunner;
40 import org.apache.zookeeper.KeeperException;
41
42 import com.google.common.util.concurrent.AtomicLongMap;
43
44 import java.io.FileNotFoundException;
45 import java.io.IOException;
46 import java.util.*;
47
48
49
50
51
52
53
54
55
56 public class DumpReplicationQueues extends Configured implements Tool {
57
58 private static final Log LOG = LogFactory.getLog(DumpReplicationQueues.class.getName());
59
60 private List<String> deadRegionServers;
61 private List<String> deletedQueues;
62 private AtomicLongMap<String> peersQueueSize;
63 private long totalSizeOfWALs;
64 private long numWalsNotFound;
65
66 public DumpReplicationQueues() {
67 deadRegionServers = new ArrayList<String>();
68 deletedQueues = new ArrayList<String>();
69 peersQueueSize = AtomicLongMap.create();
70 totalSizeOfWALs = 0;
71 numWalsNotFound = 0;
72 }
73
74 static class DumpOptions {
75 boolean hdfs = false;
76 boolean distributed = false;
77
78 public DumpOptions() {
79 }
80
81 public DumpOptions(DumpOptions that) {
82 this.hdfs = that.hdfs;
83 this.distributed = that.distributed;
84 }
85
86 boolean isHdfs () {
87 return hdfs;
88 }
89
90 boolean isDistributed() {
91 return distributed;
92 }
93
94 void setHdfs (boolean hdfs) {
95 this.hdfs = hdfs;
96 }
97
98 void setDistributed(boolean distributed) {
99 this.distributed = distributed;
100 }
101 }
102
103 static DumpOptions parseOpts(Queue<String> args) {
104 DumpOptions opts = new DumpOptions();
105
106 String cmd = null;
107 while ((cmd = args.poll()) != null) {
108 if (cmd.equals("-h") || cmd.equals("--h") || cmd.equals("--help")) {
109
110 args.add(cmd);
111 break;
112 }
113 final String hdfs = "--hdfs";
114 if (cmd.equals(hdfs)) {
115 opts.setHdfs(true);
116 continue;
117 }
118 final String distributed = "--distributed";
119 if (cmd.equals(distributed)) {
120 opts.setDistributed(true);
121 continue;
122 } else {
123 printUsageAndExit("ERROR: Unrecognized option/command: " + cmd, -1);
124 }
125
126 if (!opts.isDistributed() && opts.isHdfs()) {
127 printUsageAndExit("ERROR: --hdfs option can only be used with --distributed: " + cmd, -1);
128 }
129 }
130 return opts;
131 }
132
133
134
135
136
137
138
139 public static void main(String[] args) throws Exception {
140 Configuration conf = HBaseConfiguration.create();
141 int ret = ToolRunner.run(conf, new DumpReplicationQueues(), args);
142 System.exit(ret);
143 }
144
145 @Override
146 public int run(String[] args) throws Exception {
147
148 int errCode = -1;
149 LinkedList<String> argv = new LinkedList<String>();
150 argv.addAll(Arrays.asList(args));
151 DumpOptions opts = parseOpts(argv);
152
153
154 if (!argv.isEmpty()) {
155 errCode = 0;
156 printUsage();
157 return errCode;
158 }
159 return dumpReplicationQueues(opts);
160 }
161
162 protected void printUsage() {
163 printUsage(this.getClass().getName(), null);
164 }
165
166 protected static void printUsage(final String message) {
167 printUsage(DumpReplicationQueues.class.getName(), message);
168 }
169
170 protected static void printUsage(final String className, final String message) {
171 if (message != null && message.length() > 0) {
172 System.err.println(message);
173 }
174 System.err.println("Usage: bin/hbase " + className + " \\");
175 System.err.println(" <OPTIONS> [-D<property=value>]*");
176 System.err.println();
177 System.err.println("General Options:");
178 System.err.println(" -h|--h|--help Show this help and exit.");
179 System.err.println(" --distributed Poll each RS and print its own replication queue. "
180 + "Default only polls ZooKeeper");
181 System.err.println(" --hdfs Use HDFS to calculate usage of WALs by replication. It could be overestimated"
182 + " if replicating to multiple peers. --distributed flag is also needed.");
183 }
184
185 protected static void printUsageAndExit(final String message, final int exitCode) {
186 printUsage(message);
187 System.exit(exitCode);
188 }
189
190 private int dumpReplicationQueues(DumpOptions opts) throws Exception {
191
192 Configuration conf = getConf();
193 HBaseAdmin.checkHBaseAvailable(conf);
194 ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf);
195 ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
196
197 ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
198 new WarnOnlyAbortable(), true);
199
200 try {
201
202 LOG.info("Our Quorum: " + zkw.getQuorum());
203 List<HashMap<String, String>> replicatedTables = replicationAdmin.listReplicated();
204 if (replicatedTables.isEmpty()) {
205 LOG.info("No tables with a configured replication peer were found.");
206 return(0);
207 } else {
208 LOG.info("Replicated Tables: " + replicatedTables);
209 }
210
211 Map<String, ReplicationPeerConfig> peerConfigs = replicationAdmin.listPeerConfigs();
212
213 if (peerConfigs.isEmpty()) {
214 LOG.info("Replication is enabled but no peer configuration was found.");
215 }
216
217 System.out.println("Dumping replication peers and configurations:");
218 System.out.println(dumpPeersState(replicationAdmin, peerConfigs));
219
220 if (opts.isDistributed()) {
221 LOG.info("Found [--distributed], will poll each RegionServer.");
222 System.out.println(dumpQueues(connection, peerConfigs.keySet(), zkw, opts.isHdfs()));
223 System.out.println(dumpReplicationSummary());
224 } else {
225
226 System.out.print("Dumping replication znodes via ZooKeeper:");
227 System.out.println(ZKUtil.getReplicationZnodesDump(zkw));
228 }
229 return (0);
230 } catch (IOException e) {
231 return (-1);
232 } finally {
233 zkw.close();
234 }
235 }
236
237 public String dumpReplicationSummary() {
238 StringBuilder sb = new StringBuilder();
239 if (!deletedQueues.isEmpty()) {
240 sb.append("Found " + deletedQueues.size() + " deleted queues"
241 + ", run hbck -fixReplication in order to remove the deleted replication queues\n");
242 for (String deletedQueue : deletedQueues) {
243 sb.append(" " + deletedQueue + "\n");
244 }
245 }
246 if (!deadRegionServers.isEmpty()) {
247 sb.append("Found " + deadRegionServers.size() + " dead regionservers"
248 + ", restart one regionserver to transfer the queues of dead regionservers\n");
249 for (String deadRs : deadRegionServers) {
250 sb.append(" " + deadRs + "\n");
251 }
252 }
253 if (!peersQueueSize.isEmpty()) {
254 sb.append("Dumping all peers's number of WALs in replication queue\n");
255 for (Map.Entry<String, Long> entry : peersQueueSize.asMap().entrySet()) {
256 sb.append(" PeerId: " + entry.getKey() + " , sizeOfLogQueue: " + entry.getValue() + "\n");
257 }
258 }
259 sb.append(" Total size of WALs on HDFS: " + StringUtils.humanSize(totalSizeOfWALs) + "\n");
260 if (numWalsNotFound > 0) {
261 sb.append(" ERROR: There are " + numWalsNotFound + " WALs not found!!!\n");
262 }
263 return sb.toString();
264 }
265
266 public String dumpPeersState(ReplicationAdmin replicationAdmin,
267 Map<String, ReplicationPeerConfig> peerConfigs) throws Exception {
268 Map<String, String> currentConf;
269 StringBuilder sb = new StringBuilder();
270 for (Map.Entry<String, ReplicationPeerConfig> peer : peerConfigs.entrySet()) {
271 try {
272 ReplicationPeerConfig peerConfig = peer.getValue();
273 sb.append("Peer: " + peer.getKey() + "\n");
274 sb.append(" " + "State: "
275 + (replicationAdmin.getPeerState(peer.getKey()) ? "ENABLED" : "DISABLED") + "\n");
276 sb.append(" " + "Cluster Name: " + peerConfig.getClusterKey() + "\n");
277 currentConf = peerConfig.getConfiguration();
278
279 if (currentConf.size() > 1) {
280 sb.append(" " + "Peer Configuration: " + currentConf + "\n");
281 }
282 sb.append(" " + "Peer Table CFs: " + peerConfig.getTableCFsMap() + "\n");
283 } catch (ReplicationException re) {
284 sb.append("Got an exception while invoking ReplicationAdmin: " + re + "\n");
285 }
286 }
287 return sb.toString();
288 }
289
290 public String dumpQueues(ClusterConnection connection, Set<String> peerIds, ZooKeeperWatcher zkw,
291 boolean hdfs) throws Exception {
292 ReplicationQueuesClient queuesClient;
293 ReplicationPeers replicationPeers;
294 ReplicationQueues replicationQueues;
295 ReplicationTracker replicationTracker;
296 StringBuilder sb = new StringBuilder();
297
298 queuesClient = ReplicationFactory.getReplicationQueuesClient(zkw, getConf(), connection);
299 queuesClient.init();
300 replicationQueues = ReplicationFactory.getReplicationQueues(zkw, getConf(), connection);
301 replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection);
302 replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
303 new WarnOnlyAbortable(), new WarnOnlyStoppable());
304 List<String> liveRegionServers = replicationTracker.getListOfRegionServers();
305
306
307 try {
308 List<String> regionservers = queuesClient.getListOfReplicators();
309 if (regionservers == null || regionservers.isEmpty()) {
310 return sb.toString();
311 }
312 for (String regionserver : regionservers) {
313 List<String> queueIds = queuesClient.getAllQueues(regionserver);
314 replicationQueues.init(regionserver);
315 if (!liveRegionServers.contains(regionserver)) {
316 deadRegionServers.add(regionserver);
317 }
318 for (String queueId : queueIds) {
319 ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
320 List<String> wals = queuesClient.getLogsInQueue(regionserver, queueId);
321 Collections.sort(wals);
322 if (!peerIds.contains(queueInfo.getPeerId())) {
323 deletedQueues.add(regionserver + "/" + queueId);
324 sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
325 } else {
326 sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
327 }
328 }
329 }
330 } catch (KeeperException ke) {
331 throw new IOException(ke);
332 }
333 return sb.toString();
334 }
335
336 private String formatQueue(String regionserver, ReplicationQueues replicationQueues,
337 ReplicationQueueInfo queueInfo, String queueId, List<String> wals, boolean isDeleted,
338 boolean hdfs) throws Exception {
339 StringBuilder sb = new StringBuilder();
340 List<String> deadServers ;
341
342 sb.append("Dumping replication queue info for RegionServer: [" + regionserver + "]" + "\n");
343 sb.append(" Queue znode: " + queueId + "\n");
344 sb.append(" PeerID: " + queueInfo.getPeerId() + "\n");
345 sb.append(" Recovered: " + queueInfo.isQueueRecovered() + "\n");
346 deadServers = queueInfo.getDeadRegionServers();
347 if (deadServers.isEmpty()) {
348 sb.append(" No dead RegionServers found in this queue." + "\n");
349 } else {
350 sb.append(" Dead RegionServers: " + deadServers + "\n");
351 }
352 sb.append(" Was deleted: " + isDeleted + "\n");
353 sb.append(" Number of WALs in replication queue: " + wals.size() + "\n");
354 peersQueueSize.addAndGet(queueInfo.getPeerId(), wals.size());
355
356 for (String wal : wals) {
357 long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal);
358 sb.append(" Replication position for " + wal + ": " + (position > 0 ? position : "0"
359 + " (not started or nothing to replicate)") + "\n");
360 }
361
362 if (hdfs) {
363 FileSystem fs = FileSystem.get(getConf());
364 sb.append(" Total size of WALs on HDFS for this queue: "
365 + StringUtils.humanSize(getTotalWALSize(fs, wals, regionserver)) + "\n");
366 }
367 return sb.toString();
368 }
369
370
371
372 private long getTotalWALSize(FileSystem fs, List<String> wals, String server) throws IOException {
373 long size = 0;
374 FileStatus fileStatus;
375
376 for (String wal : wals) {
377 try {
378 fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs);
379 } catch (IOException e) {
380 if (e instanceof FileNotFoundException) {
381 numWalsNotFound++;
382 LOG.warn("WAL " + wal + " couldn't be found, skipping", e);
383 } else {
384 LOG.warn("Can't get file status of WAL " + wal + ", skipping", e);
385 }
386 continue;
387 }
388 size += fileStatus.getLen();
389 }
390
391 totalSizeOfWALs += size;
392 return size;
393 }
394
395 private static class WarnOnlyAbortable implements Abortable {
396 @Override
397 public void abort(String why, Throwable e) {
398 LOG.warn("DumpReplicationQueue received abort, ignoring. Reason: " + why);
399 if (LOG.isDebugEnabled()) {
400 LOG.debug(e);
401 }
402 }
403
404 @Override
405 public boolean isAborted() {
406 return false;
407 }
408 }
409
410 private static class WarnOnlyStoppable implements Stoppable {
411 @Override
412 public void stop(String why) {
413 LOG.warn("DumpReplicationQueue received stop, ignoring. Reason: " + why);
414 }
415
416 @Override
417 public boolean isStopped() {
418 return false;
419 }
420 }
421 }