From ec8cfe14f1a558b5da1d1987d53cccf3ce0bd5f4 Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Sat, 28 Feb 2015 19:58:30 -0600 Subject: [PATCH] HBASE-13131 ReplicationAdmin must clean up connections if constructor fails. Conflicts: hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java --- .../hadoop/hbase/client/replication/ReplicationAdmin.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index ac3bc81..2a0f0ca 100644 --- a/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ b/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -99,8 +99,17 @@ public class ReplicationAdmin implements Closeable { ZooKeeperWatcher zkw = this.connection.getZooKeeperWatcher(); try { this.replicationZk = new ReplicationZookeeper(this.connection, conf, zkw); - } catch (KeeperException e) { - throw new IOException("Unable setup the ZooKeeper connection", e); + } 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("Unable setup the ZooKeeper connection", exception); + } } } -- 1.7.10.2 (Apple Git-33)