HBASE-12837 ReplicationAdmin leaks zk connections (stack)

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
This commit is contained in:
Enis Soztutar 2015-01-16 13:16:12 -08:00
parent d0e4fb6ae3
commit 2f90bc5e20
1 changed files with 18 additions and 10 deletions

View File

@ -44,8 +44,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@ -93,11 +93,16 @@ public class ReplicationAdmin implements Closeable {
public static final String REPLICATIONGLOBAL = Integer public static final String REPLICATIONGLOBAL = Integer
.toString(HConstants.REPLICATION_SCOPE_GLOBAL); .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
private final HConnection connection; private final Connection connection;
// TODO: replication should be managed by master. All the classes except ReplicationAdmin should // TODO: replication should be managed by master. All the classes except ReplicationAdmin should
// be moved to hbase-server. Resolve it in HBASE-11392. // be moved to hbase-server. Resolve it in HBASE-11392.
private final ReplicationQueuesClient replicationQueuesClient; private final ReplicationQueuesClient replicationQueuesClient;
private final ReplicationPeers replicationPeers; 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. * Constructor that creates a connection to the local ZooKeeper ensemble.
@ -111,8 +116,8 @@ public class ReplicationAdmin implements Closeable {
throw new RuntimeException("hbase.replication isn't true, please " + throw new RuntimeException("hbase.replication isn't true, please " +
"enable it in order to use replication"); "enable it in order to use replication");
} }
this.connection = HConnectionManager.getConnection(conf); this.connection = ConnectionFactory.createConnection(conf);
ZooKeeperWatcher zkw = createZooKeeperWatcher(); zkw = createZooKeeperWatcher();
try { try {
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection); this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
this.replicationPeers.init(); this.replicationPeers.init();
@ -126,19 +131,19 @@ public class ReplicationAdmin implements Closeable {
} }
private ZooKeeperWatcher createZooKeeperWatcher() throws IOException { private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
return new ZooKeeperWatcher(connection.getConfiguration(), // This Abortable doesn't 'abort'... it just logs.
"Replication Admin", new Abortable() { return new ZooKeeperWatcher(connection.getConfiguration(), "ReplicationAdmin", new Abortable() {
@Override @Override
public void abort(String why, Throwable e) { public void abort(String why, Throwable e) {
LOG.error(why, 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 @Override
public boolean isAborted() { public boolean isAborted() {
return false; return false;
} }
}); });
} }
@ -451,6 +456,9 @@ public class ReplicationAdmin implements Closeable {
@Override @Override
public void close() throws IOException { public void close() throws IOException {
if (this.zkw != null) {
this.zkw.close();
}
if (this.connection != null) { if (this.connection != null) {
this.connection.close(); this.connection.close();
} }
@ -471,7 +479,7 @@ public class ReplicationAdmin implements Closeable {
public List<HashMap<String, String>> listReplicated() throws IOException { public List<HashMap<String, String>> listReplicated() throws IOException {
List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>(); List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
Admin admin = new HBaseAdmin(this.connection.getConfiguration()); Admin admin = connection.getAdmin();
HTableDescriptor[] tables; HTableDescriptor[] tables;
try { try {
tables = admin.listTables(); tables = admin.listTables();