hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bus...@apache.org
Subject hbase git commit: HBASE-13131 ReplicationAdmin must clean up connections if constructor fails.
Date Sun, 01 Mar 2015 06:26:06 GMT
Repository: hbase
Updated Branches:
  refs/heads/branch-1 5b0b43b48 -> 1efd29e34


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/1efd29e3
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1efd29e3
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1efd29e3

Branch: refs/heads/branch-1
Commit: 1efd29e3433925bd51ecd9f8e664acb4bff0d8af
Parents: 5b0b43b
Author: Sean Busbey <busbey@apache.org>
Authored: Sat Feb 28 19:58:30 2015 -0600
Committer: Sean Busbey <busbey@apache.org>
Committed: Sun Mar 1 00:24:04 2015 -0600

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    | 33 ++++++++++++++------
 1 file changed, 24 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1efd29e3/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 1a04308..1bb18b2 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);
+      }
     }
   }
 


Mime
View raw message