Repository: hbase Updated Branches: refs/heads/master dd78f459e -> dad2474f0
HBASE-13131 ReplicationAdmin must clean up connections if constructor fails. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/dad2474f Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/dad2474f Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/dad2474f Branch: refs/heads/master Commit: dad2474f08d201d09989e36f5cf1c25d3fa4acee Parents: dd78f45 Author: Sean Busbey <bus...@apache.org> Authored: Sat Feb 28 19:58:30 2015 -0600 Committer: Sean Busbey <bus...@apache.org> Committed: Sun Mar 1 00:20:44 2015 -0600 ---------------------------------------------------------------------- .../client/replication/ReplicationAdmin.java | 33 ++++++++++++++------ 1 file changed, 24 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/dad2474f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index ca66fb3..c8b8b47 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -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); + } } }