hbase git commit: HBASE-13131 ReplicationAdmin must clean up connections if constructor fails.
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); + } } }
hbase git commit: HBASE-13131 ReplicationAdmin must clean up connections if constructor fails.
Repository: hbase Updated Branches: refs/heads/branch-1.0 b237f792c - 9769fb6d9 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/9769fb6d Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9769fb6d Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9769fb6d Branch: refs/heads/branch-1.0 Commit: 9769fb6d94d2ba3e41b1ff288d15a9a6d32ea9df Parents: b237f79 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:06:21 2015 -0600 -- .../client/replication/ReplicationAdmin.java| 33 ++-- 1 file changed, 24 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9769fb6d/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 cd56261..2ada823 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 @@ -117,16 +117,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); + } } }
hbase git commit: HBASE-13131 ReplicationAdmin must clean up connections if constructor fails.
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); + } } }
hbase git commit: HBASE-13131 ReplicationAdmin must clean up connections if constructor fails.
Repository: hbase Updated Branches: refs/heads/0.94 71be2f036 - ec8cfe14f 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/ec8cfe14 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/ec8cfe14 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/ec8cfe14 Branch: refs/heads/0.94 Commit: ec8cfe14f1a558b5da1d1987d53cccf3ce0bd5f4 Parents: 71be2f0 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:43:00 2015 -0600 -- .../hbase/client/replication/ReplicationAdmin.java | 13 +++-- 1 file changed, 11 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/ec8cfe14/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java -- 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); + } } }