HBASE-16947 Some improvements for DumpReplicationQueues tool
Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
parent
d0e61b0e9a
commit
7b74dd0374
|
@ -51,7 +51,9 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
|
||||||
@Override
|
@Override
|
||||||
public void init() throws ReplicationException {
|
public void init() throws ReplicationException {
|
||||||
try {
|
try {
|
||||||
ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
|
if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) {
|
||||||
|
ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
|
||||||
|
}
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
throw new ReplicationException("Internal error while initializing a queues client", e);
|
throw new ReplicationException("Internal error while initializing a queues client", e);
|
||||||
}
|
}
|
||||||
|
|
|
@ -82,14 +82,18 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
||||||
public void init(String serverName) throws ReplicationException {
|
public void init(String serverName) throws ReplicationException {
|
||||||
this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName);
|
this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName);
|
||||||
try {
|
try {
|
||||||
ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
|
if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) {
|
||||||
|
ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
|
||||||
|
}
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
throw new ReplicationException("Could not initialize replication queues.", e);
|
throw new ReplicationException("Could not initialize replication queues.", e);
|
||||||
}
|
}
|
||||||
if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
|
if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
|
||||||
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
|
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
|
||||||
try {
|
try {
|
||||||
ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
|
if (ZKUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) {
|
||||||
|
ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
|
||||||
|
}
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
throw new ReplicationException("Could not initialize hfile references replication queue.",
|
throw new ReplicationException("Could not initialize hfile references replication queue.",
|
||||||
e);
|
e);
|
||||||
|
|
|
@ -40,6 +40,9 @@ import org.apache.hadoop.util.ToolRunner;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
import org.mortbay.util.IO;
|
import org.mortbay.util.IO;
|
||||||
|
|
||||||
|
import com.google.common.util.concurrent.AtomicLongMap;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
|
|
||||||
|
@ -55,6 +58,20 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(DumpReplicationQueues.class.getName());
|
private static final Log LOG = LogFactory.getLog(DumpReplicationQueues.class.getName());
|
||||||
|
|
||||||
|
private List<String> deadRegionServers;
|
||||||
|
private List<String> deletedQueues;
|
||||||
|
private AtomicLongMap<String> peersQueueSize;
|
||||||
|
private long totalSizeOfWALs;
|
||||||
|
private long numWalsNotFound;
|
||||||
|
|
||||||
|
public DumpReplicationQueues() {
|
||||||
|
deadRegionServers = new ArrayList<String>();
|
||||||
|
deletedQueues = new ArrayList<String>();
|
||||||
|
peersQueueSize = AtomicLongMap.create();
|
||||||
|
totalSizeOfWALs = 0;
|
||||||
|
numWalsNotFound = 0;
|
||||||
|
}
|
||||||
|
|
||||||
static class DumpOptions {
|
static class DumpOptions {
|
||||||
boolean hdfs = false;
|
boolean hdfs = false;
|
||||||
boolean distributed = false;
|
boolean distributed = false;
|
||||||
|
@ -155,14 +172,16 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
if (message != null && message.length() > 0) {
|
if (message != null && message.length() > 0) {
|
||||||
System.err.println(message);
|
System.err.println(message);
|
||||||
}
|
}
|
||||||
System.err.println("Usage: java " + className + " \\");
|
System.err.println("Usage: bin/hbase " + className + " \\");
|
||||||
System.err.println(" <OPTIONS> [-D<property=value>]*");
|
System.err.println(" <OPTIONS> [-D<property=value>]*");
|
||||||
System.err.println();
|
System.err.println();
|
||||||
System.err.println("General Options:");
|
System.err.println("General Options:");
|
||||||
System.err.println(" distributed Poll each RS and print its own replication queue. "
|
System.err.println(" -h|--h|--help Show this help and exit.");
|
||||||
|
System.err.println(" --distributed Poll each RS and print its own replication queue. "
|
||||||
+ "Default only polls ZooKeeper");
|
+ "Default only polls ZooKeeper");
|
||||||
System.err.println(" hdfs Use HDFS to calculate usage of WALs by replication. It could be overestimated"
|
System.err.println(" --hdfs Use HDFS to calculate usage of WALs by replication."
|
||||||
+ " if replicating to multiple peers. --distributed flag is also needed.");
|
+ " It could be overestimated if replicating to multiple peers."
|
||||||
|
+ " --distributed flag is also needed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static void printUsageAndExit(final String message, final int exitCode) {
|
protected static void printUsageAndExit(final String message, final int exitCode) {
|
||||||
|
@ -176,7 +195,6 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
HBaseAdmin.available(conf);
|
HBaseAdmin.available(conf);
|
||||||
ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf);
|
ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf);
|
||||||
ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
|
ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
|
||||||
long deleted = 0;
|
|
||||||
|
|
||||||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
|
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
|
||||||
new WarnOnlyAbortable(), true);
|
new WarnOnlyAbortable(), true);
|
||||||
|
@ -203,11 +221,8 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
|
|
||||||
if (opts.isDistributed()) {
|
if (opts.isDistributed()) {
|
||||||
LOG.info("Found [--distributed], will poll each RegionServer.");
|
LOG.info("Found [--distributed], will poll each RegionServer.");
|
||||||
System.out.println(dumpQueues(connection, zkw, opts.isHdfs(), deleted));
|
System.out.println(dumpQueues(connection, zkw, peerConfigs.keySet(), opts.isHdfs()));
|
||||||
if (deleted > 0) {
|
System.out.println(dumpReplicationSummary());
|
||||||
LOG.warn("Found " + deleted +" deleted queues"
|
|
||||||
+ ", run hbck -fixReplication in order to remove the deleted replication queues");
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
// use ZK instead
|
// use ZK instead
|
||||||
System.out.print("Dumping replication znodes via ZooKeeper:");
|
System.out.print("Dumping replication znodes via ZooKeeper:");
|
||||||
|
@ -221,21 +236,54 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public String dumpReplicationSummary() {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
if (!deletedQueues.isEmpty()) {
|
||||||
|
sb.append("Found " + deletedQueues.size() + " deleted queues"
|
||||||
|
+ ", run hbck -fixReplication in order to remove the deleted replication queues\n");
|
||||||
|
for (String deletedQueue : deletedQueues) {
|
||||||
|
sb.append(" " + deletedQueue + "\n");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!deadRegionServers.isEmpty()) {
|
||||||
|
sb.append("Found " + deadRegionServers.size() + " dead regionservers"
|
||||||
|
+ ", restart one regionserver to transfer the queues of dead regionservers\n");
|
||||||
|
for (String deadRs : deadRegionServers) {
|
||||||
|
sb.append(" " + deadRs + "\n");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!peersQueueSize.isEmpty()) {
|
||||||
|
sb.append("Dumping all peers's number of WALs in replication queue\n");
|
||||||
|
for (Map.Entry<String, Long> entry : peersQueueSize.asMap().entrySet()) {
|
||||||
|
sb.append(" PeerId: " + entry.getKey() + " , sizeOfLogQueue: " + entry.getValue() + "\n");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
sb.append(" Total size of WALs on HDFS: " + StringUtils.humanSize(totalSizeOfWALs) + "\n");
|
||||||
|
if (numWalsNotFound > 0) {
|
||||||
|
sb.append(" ERROR: There are " + numWalsNotFound + " WALs not found!!!\n");
|
||||||
|
}
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
public String dumpPeersState(ReplicationAdmin replicationAdmin,
|
public String dumpPeersState(ReplicationAdmin replicationAdmin,
|
||||||
Map<String, ReplicationPeerConfig> peerConfigs) throws Exception {
|
Map<String, ReplicationPeerConfig> peerConfigs) throws Exception {
|
||||||
Map<String, String> currentConf;
|
Map<String, String> currentConf;
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
for (Map.Entry<String, ReplicationPeerConfig> peer : peerConfigs.entrySet()) {
|
for (Map.Entry<String, ReplicationPeerConfig> peer : peerConfigs.entrySet()) {
|
||||||
try {
|
try {
|
||||||
|
ReplicationPeerConfig peerConfig = peer.getValue();
|
||||||
sb.append("Peer: " + peer.getKey() + "\n");
|
sb.append("Peer: " + peer.getKey() + "\n");
|
||||||
sb.append(" " + "State: " + (replicationAdmin.getPeerState(peer.getKey()) ? "ENABLED" : "DISABLED") + "\n");
|
sb.append(" " + "State: "
|
||||||
sb.append(" " + "Cluster Name: " + peer.getValue() + "\n");
|
+ (replicationAdmin.getPeerState(peer.getKey()) ? "ENABLED" : "DISABLED") + "\n");
|
||||||
currentConf = peer.getValue().getConfiguration();
|
sb.append(" " + "Cluster Name: " + peerConfig.getClusterKey() + "\n");
|
||||||
|
sb.append(" " + "Replication Endpoint: " + peerConfig.getReplicationEndpointImpl() + "\n");
|
||||||
|
currentConf = peerConfig.getConfiguration();
|
||||||
// Only show when we have a custom configuration for the peer
|
// Only show when we have a custom configuration for the peer
|
||||||
if (currentConf.size() > 1) {
|
if (currentConf.size() > 1) {
|
||||||
sb.append(" " + "Peer Configuration: " + currentConf + "\n");
|
sb.append(" " + "Peer Configuration: " + currentConf + "\n");
|
||||||
}
|
}
|
||||||
sb.append(" " + "Peer Table CFs: " + replicationAdmin.getPeerTableCFs(peer.getKey()) + "\n");
|
sb.append(" " + "Peer Table CFs: " + peerConfig.getTableCFsMap() + "\n");
|
||||||
|
sb.append(" " + "Peer Namespaces: " + peerConfig.getNamespaces() + "\n");
|
||||||
} catch (ReplicationException re) {
|
} catch (ReplicationException re) {
|
||||||
sb.append("Got an exception while invoking ReplicationAdmin: " + re + "\n");
|
sb.append("Got an exception while invoking ReplicationAdmin: " + re + "\n");
|
||||||
}
|
}
|
||||||
|
@ -243,11 +291,12 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, boolean hdfs, long deleted)
|
public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, Set<String> peerIds,
|
||||||
throws Exception {
|
boolean hdfs) throws Exception {
|
||||||
ReplicationQueuesClient queuesClient;
|
ReplicationQueuesClient queuesClient;
|
||||||
ReplicationPeers replicationPeers;
|
ReplicationPeers replicationPeers;
|
||||||
ReplicationQueues replicationQueues;
|
ReplicationQueues replicationQueues;
|
||||||
|
ReplicationTracker replicationTracker;
|
||||||
ReplicationQueuesClientArguments replicationArgs =
|
ReplicationQueuesClientArguments replicationArgs =
|
||||||
new ReplicationQueuesClientArguments(getConf(), new WarnOnlyAbortable(), zkw);
|
new ReplicationQueuesClientArguments(getConf(), new WarnOnlyAbortable(), zkw);
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
|
@ -256,24 +305,29 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
queuesClient.init();
|
queuesClient.init();
|
||||||
replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs);
|
replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs);
|
||||||
replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection);
|
replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection);
|
||||||
replicationPeers.init();
|
replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
|
||||||
|
new WarnOnlyAbortable(), new WarnOnlyStoppable());
|
||||||
|
List<String> liveRegionServers = replicationTracker.getListOfRegionServers();
|
||||||
|
|
||||||
// Loops each peer on each RS and dumps the queues
|
// Loops each peer on each RS and dumps the queues
|
||||||
|
|
||||||
Set<String> peerIds = new HashSet<String>(replicationPeers.getAllPeerIds());
|
|
||||||
try {
|
try {
|
||||||
List<String> regionservers = queuesClient.getListOfReplicators();
|
List<String> regionservers = queuesClient.getListOfReplicators();
|
||||||
for (String regionserver : regionservers) {
|
for (String regionserver : regionservers) {
|
||||||
List<String> queueIds = queuesClient.getAllQueues(regionserver);
|
List<String> queueIds = queuesClient.getAllQueues(regionserver);
|
||||||
replicationQueues.init(regionserver);
|
replicationQueues.init(regionserver);
|
||||||
|
if (!liveRegionServers.contains(regionserver)) {
|
||||||
|
deadRegionServers.add(regionserver);
|
||||||
|
}
|
||||||
for (String queueId : queueIds) {
|
for (String queueId : queueIds) {
|
||||||
ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
|
ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
|
||||||
List<String> wals = queuesClient.getLogsInQueue(regionserver, queueId);
|
List<String> wals = queuesClient.getLogsInQueue(regionserver, queueId);
|
||||||
if (!peerIds.contains(queueInfo.getPeerId())) {
|
if (!peerIds.contains(queueInfo.getPeerId())) {
|
||||||
deleted++;
|
deletedQueues.add(regionserver + "/" + queueId);
|
||||||
sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
|
sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true,
|
||||||
|
hdfs));
|
||||||
} else {
|
} else {
|
||||||
sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
|
sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false,
|
||||||
|
hdfs));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -291,6 +345,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
List<String> deadServers ;
|
List<String> deadServers ;
|
||||||
|
|
||||||
sb.append("Dumping replication queue info for RegionServer: [" + regionserver + "]" + "\n");
|
sb.append("Dumping replication queue info for RegionServer: [" + regionserver + "]" + "\n");
|
||||||
|
sb.append(" Queue znode: " + queueId + "\n");
|
||||||
sb.append(" PeerID: " + queueInfo.getPeerId() + "\n");
|
sb.append(" PeerID: " + queueInfo.getPeerId() + "\n");
|
||||||
sb.append(" Recovered: " + queueInfo.isQueueRecovered() + "\n");
|
sb.append(" Recovered: " + queueInfo.isQueueRecovered() + "\n");
|
||||||
deadServers = queueInfo.getDeadRegionServers();
|
deadServers = queueInfo.getDeadRegionServers();
|
||||||
|
@ -301,6 +356,8 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
sb.append(" Was deleted: " + isDeleted + "\n");
|
sb.append(" Was deleted: " + isDeleted + "\n");
|
||||||
sb.append(" Number of WALs in replication queue: " + wals.size() + "\n");
|
sb.append(" Number of WALs in replication queue: " + wals.size() + "\n");
|
||||||
|
peersQueueSize.addAndGet(queueInfo.getPeerId(), wals.size());
|
||||||
|
|
||||||
for (String wal : wals) {
|
for (String wal : wals) {
|
||||||
long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal);
|
long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal);
|
||||||
sb.append(" Replication position for " + wal + ": " + (position > 0 ? position : "0"
|
sb.append(" Replication position for " + wal + ": " + (position > 0 ? position : "0"
|
||||||
|
@ -325,11 +382,18 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
try {
|
try {
|
||||||
fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs);
|
fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.warn("WAL " + wal + " couldn't be found, skipping");
|
if (e instanceof FileNotFoundException) {
|
||||||
break;
|
numWalsNotFound++;
|
||||||
|
LOG.warn("WAL " + wal + " couldn't be found, skipping", e);
|
||||||
|
} else {
|
||||||
|
LOG.warn("Can't get file status of WAL " + wal + ", skipping", e);
|
||||||
|
}
|
||||||
|
continue;
|
||||||
}
|
}
|
||||||
size += fileStatus.getLen();
|
size += fileStatus.getLen();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
totalSizeOfWALs += size;
|
||||||
return size;
|
return size;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -348,9 +412,15 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void usage(final String errorMsg) {
|
private static class WarnOnlyStoppable implements Stoppable {
|
||||||
if (errorMsg != null && errorMsg.length() > 0) {
|
@Override
|
||||||
LOG.error(errorMsg);
|
public void stop(String why) {
|
||||||
|
LOG.warn("DumpReplicationQueue received stop, ignoring. Reason: " + why);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isStopped() {
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue