HBASE-13131 ReplicationAdmin must clean up connections if constructor fails.

This commit is contained in:
Sean Busbey 2015-02-28 19:58:30 -06:00
parent dd78f459e8
commit dad2474f08
1 changed files with 24 additions and 9 deletions

View File

@ -122,16 +122,31 @@ public class ReplicationAdmin implements Closeable {
"enable it in order to use replication");
}
this.connection = ConnectionFactory.createConnection(conf);
zkw = createZooKeeperWatcher();
try {
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
this.replicationPeers.init();
this.replicationQueuesClient =
ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
this.replicationQueuesClient.init();
} catch (ReplicationException e) {
throw new IOException("Error initializing the replication admin client.", e);
zkw = createZooKeeperWatcher();
try {
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
this.replicationPeers.init();
this.replicationQueuesClient =
ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
this.replicationQueuesClient.init();
} catch (Exception exception) {
if (zkw != null) {
zkw.close();
}
throw exception;
}
} catch (Exception exception) {
if (connection != null) {
connection.close();
}
if (exception instanceof IOException) {
throw (IOException) exception;
} else if (exception instanceof RuntimeException) {
throw (RuntimeException) exception;
} else {
throw new IOException("Error initializing the replication admin client.", exception);
}
}
}