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 1a043082433..1bb18b2b8e2 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 @@ -125,16 +125,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); + } } }