Repository: hbase Updated Branches: refs/heads/0.98 bcf14c843 -> 28032b3da
HBASE-13131 ReplicationAdmin must clean up connections if constructor fails. Conflicts: hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/28032b3d Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/28032b3d Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/28032b3d Branch: refs/heads/0.98 Commit: 28032b3daa720caac034c01187d7096b0d9bc390 Parents: bcf14c8 Author: Sean Busbey <bus...@apache.org> Authored: Sat Feb 28 19:58:30 2015 -0600 Committer: Sean Busbey <bus...@apache.org> Committed: Sat Feb 28 20:09:24 2015 -0600 ---------------------------------------------------------------------- .../client/replication/ReplicationAdmin.java | 33 ++++++++++++++------ 1 file changed, 24 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/28032b3d/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 2e0b5b8..e05a51d 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 @@ -109,16 +109,31 @@ public class ReplicationAdmin implements Closeable { "enable it in order to use replication"); } this.connection = HConnectionManager.getConnection(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); + } } }