hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e...@apache.org
Subject hbase git commit: HBASE-12837 ReplicationAdmin leaks zk connections (stack)
Date Fri, 16 Jan 2015 21:16:28 GMT
Repository: hbase
Updated Branches:
  refs/heads/master af725a035 -> 092c91eb0


HBASE-12837 ReplicationAdmin leaks zk connections (stack)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/092c91eb
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/092c91eb
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/092c91eb

Branch: refs/heads/master
Commit: 092c91eb0fc2a6b4044183e9ece71dd03711045d
Parents: af725a0
Author: Enis Soztutar <enis@apache.org>
Authored: Fri Jan 16 13:16:12 2015 -0800
Committer: Enis Soztutar <enis@apache.org>
Committed: Fri Jan 16 13:16:12 2015 -0800

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java      | 18 +++++++++++++-----
 1 file changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/092c91eb/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 ea7dc60..2d5c5e9 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
@@ -95,6 +95,11 @@ public class ReplicationAdmin implements Closeable {
   // be moved to hbase-server. Resolve it in HBASE-11392.
   private final ReplicationQueuesClient replicationQueuesClient;
   private final ReplicationPeers replicationPeers;
+  /**
+   * A watcher used by replicationPeers and replicationQueuesClient. Keep reference so can
dispose
+   * on {@link #close()}.
+   */
+  private final ZooKeeperWatcher zkw;
 
   /**
    * Constructor that creates a connection to the local ZooKeeper ensemble.
@@ -109,7 +114,7 @@ public class ReplicationAdmin implements Closeable {
           "enable it in order to use replication");
     }
     this.connection = ConnectionFactory.createConnection(conf);
-    ZooKeeperWatcher zkw = createZooKeeperWatcher();
+    zkw = createZooKeeperWatcher();
     try {
       this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
       this.replicationPeers.init();
@@ -123,19 +128,19 @@ public class ReplicationAdmin implements Closeable {
   }
 
   private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
-    return new ZooKeeperWatcher(connection.getConfiguration(),
-      "Replication Admin", new Abortable() {
+    // This Abortable doesn't 'abort'... it just logs.
+    return new ZooKeeperWatcher(connection.getConfiguration(), "ReplicationAdmin", new Abortable()
{
       @Override
       public void abort(String why, Throwable e) {
         LOG.error(why, e);
-        System.exit(1);
+        // We used to call system.exit here but this script can be embedded by other programs
that
+        // want to do replication stuff... so inappropriate calling System.exit. Just log
for now.
       }
 
       @Override
       public boolean isAborted() {
         return false;
       }
-
     });
   }
 
@@ -448,6 +453,9 @@ public class ReplicationAdmin implements Closeable {
 
   @Override
   public void close() throws IOException {
+    if (this.zkw != null) {
+      this.zkw.close();
+    }
     if (this.connection != null) {
       this.connection.close();
     }


Mime
View raw message