You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/07/29 07:55:54 UTC

[GitHub] [hbase] sandeepvinayak opened a new pull request #2162: HBASE-24788: Fix the connection leaks in hbase

sandeepvinayak opened a new pull request #2162:
URL: https://github.com/apache/hbase/pull/2162


   Fixing the connection leaks in hbase due to closing hbase admin without closing connection. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2162: HBASE-24788: Fix the connection leaks in hbase

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2162:
URL: https://github.com/apache/hbase/pull/2162#issuecomment-665287489






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2162: HBASE-24788: Fix the connection leaks on getting hbase admin from unclosed connection

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2162:
URL: https://github.com/apache/hbase/pull/2162#issuecomment-665505722






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] sandeepvinayak commented on a change in pull request #2162: HBASE-24788: Fix the connection leaks on getting hbase admin from unclosed connection

Posted by GitBox <gi...@apache.org>.
sandeepvinayak commented on a change in pull request #2162:
URL: https://github.com/apache/hbase/pull/2162#discussion_r462140532



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
##########
@@ -161,25 +162,24 @@ public static void setupRegionReplicaReplication(Configuration conf) throws IOEx
     if (!isRegionReplicaReplicationEnabled(conf)) {
       return;
     }
-    Admin admin = ConnectionFactory.createConnection(conf).getAdmin();
-    ReplicationPeerConfig peerConfig = null;
-    try {
+
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+      Admin admin = connection.getAdmin()) {
+      ReplicationPeerConfig peerConfig = null;
       peerConfig = admin.getReplicationPeerConfig(REGION_REPLICA_REPLICATION_PEER);
-    } catch (ReplicationPeerNotFoundException e) {
-      LOG.warn("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER
-          + " not exist", e);
-    }
-    try {
+
       if (peerConfig == null) {
         LOG.info("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER
-            + " not exist. Creating...");
+          + " not exist. Creating...");
         peerConfig = new ReplicationPeerConfig();
         peerConfig.setClusterKey(ZKConfig.getZooKeeperClusterKey(conf));
         peerConfig.setReplicationEndpointImpl(RegionReplicaReplicationEndpoint.class.getName());
         admin.addReplicationPeer(REGION_REPLICA_REPLICATION_PEER, peerConfig);
       }
-    } finally {
-      admin.close();
+    } catch (ReplicationPeerNotFoundException e) {
+      LOG.warn(
+        "Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER + " not exist",
+        e);

Review comment:
       yeah, I was about to submit the change for this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] bharathv merged pull request #2162: HBASE-24788: Fix the connection leaks on getting hbase admin from unclosed connection

Posted by GitBox <gi...@apache.org>.
bharathv merged pull request #2162:
URL: https://github.com/apache/hbase/pull/2162


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] sandeepvinayak commented on pull request #2162: HBASE-24788: Fix the connection leaks in hbase

Posted by GitBox <gi...@apache.org>.
sandeepvinayak commented on pull request #2162:
URL: https://github.com/apache/hbase/pull/2162#issuecomment-665267925


   @bharathv  @apurtell @xcangCRM Can you please review this ?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2162: HBASE-24788: Fix the connection leaks on getting hbase admin from unclosed connection

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2162:
URL: https://github.com/apache/hbase/pull/2162#discussion_r462138788



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
##########
@@ -161,25 +162,24 @@ public static void setupRegionReplicaReplication(Configuration conf) throws IOEx
     if (!isRegionReplicaReplicationEnabled(conf)) {
       return;
     }
-    Admin admin = ConnectionFactory.createConnection(conf).getAdmin();
-    ReplicationPeerConfig peerConfig = null;
-    try {
+
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+      Admin admin = connection.getAdmin()) {
+      ReplicationPeerConfig peerConfig = null;
       peerConfig = admin.getReplicationPeerConfig(REGION_REPLICA_REPLICATION_PEER);
-    } catch (ReplicationPeerNotFoundException e) {
-      LOG.warn("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER
-          + " not exist", e);
-    }
-    try {
+
       if (peerConfig == null) {
         LOG.info("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER
-            + " not exist. Creating...");
+          + " not exist. Creating...");
         peerConfig = new ReplicationPeerConfig();
         peerConfig.setClusterKey(ZKConfig.getZooKeeperClusterKey(conf));
         peerConfig.setReplicationEndpointImpl(RegionReplicaReplicationEndpoint.class.getName());
         admin.addReplicationPeer(REGION_REPLICA_REPLICATION_PEER, peerConfig);
       }
-    } finally {
-      admin.close();
+    } catch (ReplicationPeerNotFoundException e) {
+      LOG.warn(
+        "Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER + " not exist",
+        e);

Review comment:
       If we catch this Exception, don't we want to create replication peer id?
   We can model this with:
   ```
   try(closeables){
     try{
       peerConfig = admin.getReplicationPeerConfig(REGION_REPLICA_REPLICATION_PEER);
     } catch (ReplicationPeerNotFoundException e){
     }
     if (peerConfig == null) {
       .....
     }
   }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] bharathv commented on a change in pull request #2162: HBASE-24788: Fix the connection leaks in hbase

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2162:
URL: https://github.com/apache/hbase/pull/2162#discussion_r461865026



##########
File path: hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
##########
@@ -155,11 +155,10 @@ public void write(KEY key, Mutation value)
    * @param context  The current task context.
    * @return The newly created writer instance.
    * @throws IOException When creating the writer fails.
-   * @throws InterruptedException When the jobs is cancelled.
    */
   @Override
   public RecordWriter<KEY, Mutation> getRecordWriter(TaskAttemptContext context)
-  throws IOException, InterruptedException {
+  throws IOException {

Review comment:
       nit: Check-style might flag this for bad indentation (needs nested indent)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
##########
@@ -161,25 +162,27 @@ public static void setupRegionReplicaReplication(Configuration conf) throws IOEx
     if (!isRegionReplicaReplicationEnabled(conf)) {
       return;
     }
-    Admin admin = ConnectionFactory.createConnection(conf).getAdmin();
-    ReplicationPeerConfig peerConfig = null;
-    try {
-      peerConfig = admin.getReplicationPeerConfig(REGION_REPLICA_REPLICATION_PEER);
-    } catch (ReplicationPeerNotFoundException e) {
-      LOG.warn("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER
-          + " not exist", e);
-    }
-    try {
+
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+      Admin admin = connection.getAdmin()) {
+

Review comment:
       nit: remove extra newline

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
##########
@@ -161,25 +162,27 @@ public static void setupRegionReplicaReplication(Configuration conf) throws IOEx
     if (!isRegionReplicaReplicationEnabled(conf)) {
       return;
     }
-    Admin admin = ConnectionFactory.createConnection(conf).getAdmin();
-    ReplicationPeerConfig peerConfig = null;
-    try {
-      peerConfig = admin.getReplicationPeerConfig(REGION_REPLICA_REPLICATION_PEER);
-    } catch (ReplicationPeerNotFoundException e) {
-      LOG.warn("Region replica replication peer id=" + REGION_REPLICA_REPLICATION_PEER
-          + " not exist", e);
-    }
-    try {
+
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+      Admin admin = connection.getAdmin()) {
+
+      ReplicationPeerConfig peerConfig = null;
+      try {
+        peerConfig = admin.getReplicationPeerConfig(REGION_REPLICA_REPLICATION_PEER);
+      } catch (ReplicationPeerNotFoundException e) {
+        LOG.warn(

Review comment:
       nit: can be condensed to fewer lines.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
##########
@@ -149,8 +150,8 @@ public void testClearBlockCacheFromAdmin() throws Exception {
 
   @Test
   public void testClearBlockCacheFromAsyncAdmin() throws Exception {
-    AsyncAdmin admin =
-        ConnectionFactory.createAsyncConnection(HTU.getConfiguration()).get().getAdmin();
+    AsyncConnection conn = ConnectionFactory.createAsyncConnection(HTU.getConfiguration()).get();
+    AsyncAdmin admin = conn.getAdmin();

Review comment:
       this should be a try-with-resources too?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on pull request #2162: HBASE-24788: Fix the connection leaks on getting hbase admin from unclosed connection

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2162:
URL: https://github.com/apache/hbase/pull/2162#issuecomment-665521872


   Changes look good, +1.
   Test failures seem relevant? Failures are present for the updated tests. Triggered another build, let's wait for it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org