commit 008cf79545c017a4103adcc2d5e8bfa4868a8df7 Author: Enis Soztutar Date: Fri Jan 16 13:16:12 2015 -0800 HBASE-12837 ReplicationAdmin leaks zk connections (stack) Conflicts: hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index 6e5eb2e..2e0b5b8 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -90,6 +90,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. @@ -104,7 +109,7 @@ public class ReplicationAdmin implements Closeable { "enable it in order to use replication"); } this.connection = HConnectionManager.getConnection(conf); - ZooKeeperWatcher zkw = createZooKeeperWatcher(); + zkw = createZooKeeperWatcher(); try { this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection); this.replicationPeers.init(); @@ -118,19 +123,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; } - }); } @@ -289,6 +294,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(); }