HBASE-12837 ReplicationAdmin leaks zk connections (stack)

This commit is contained in:
Enis Soztutar 2015-01-16 13:16:12 -08:00
parent af725a0357
commit 092c91eb0f
1 changed files with 13 additions and 5 deletions

View File

@ -95,6 +95,11 @@ public class ReplicationAdmin implements Closeable {
// be moved to hbase-server. Resolve it in HBASE-11392.
private final ReplicationQueuesClient replicationQueuesClient;
private final ReplicationPeers replicationPeers;
/**
* A watcher used by replicationPeers and replicationQueuesClient. Keep reference so can dispose
* on {@link #close()}.
*/
private final ZooKeeperWatcher zkw;
/**
* Constructor that creates a connection to the local ZooKeeper ensemble.
@ -109,7 +114,7 @@ public class ReplicationAdmin implements Closeable {
"enable it in order to use replication");
}
this.connection = ConnectionFactory.createConnection(conf);
ZooKeeperWatcher zkw = createZooKeeperWatcher();
zkw = createZooKeeperWatcher();
try {
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
this.replicationPeers.init();
@ -123,19 +128,19 @@ public class ReplicationAdmin implements Closeable {
}
private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
return new ZooKeeperWatcher(connection.getConfiguration(),
"Replication Admin", new Abortable() {
// This Abortable doesn't 'abort'... it just logs.
return new ZooKeeperWatcher(connection.getConfiguration(), "ReplicationAdmin", new Abortable() {
@Override
public void abort(String why, Throwable e) {
LOG.error(why, e);
System.exit(1);
// We used to call system.exit here but this script can be embedded by other programs that
// want to do replication stuff... so inappropriate calling System.exit. Just log for now.
}
@Override
public boolean isAborted() {
return false;
}
});
}
@ -448,6 +453,9 @@ public class ReplicationAdmin implements Closeable {
@Override
public void close() throws IOException {
if (this.zkw != null) {
this.zkw.close();
}
if (this.connection != null) {
this.connection.close();
}