hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject hbase git commit: HBASE-14361 Investigate unused connection objects
Date Thu, 10 Sep 2015 16:19:16 GMT
Repository: hbase
Updated Branches:
  refs/heads/branch-1.2 26674463e -> 17a59cb7e


HBASE-14361 Investigate unused connection objects

Signed-off-by: stack <stack@apache.org>


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

Branch: refs/heads/branch-1.2
Commit: 17a59cb7eb68cd62f5944360b7acfb3233314ca0
Parents: 2667446
Author: chenheng <chenheng@fenbi.com>
Authored: Wed Sep 9 15:40:22 2015 +0800
Committer: stack <stack@apache.org>
Committed: Thu Sep 10 09:19:05 2015 -0700

----------------------------------------------------------------------
 .../hbase/replication/regionserver/ReplicationSink.java   | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/17a59cb7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 7d47677..11019f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -72,7 +72,7 @@ public class ReplicationSink {
 
   private static final Log LOG = LogFactory.getLog(ReplicationSink.class);
   private final Configuration conf;
-  private final Connection sharedHtableCon;
+  private Connection sharedHtableCon;
   private final MetricsSink metrics;
   private final AtomicLong totalReplicatedEdits = new AtomicLong();
 
@@ -88,7 +88,6 @@ public class ReplicationSink {
     this.conf = HBaseConfiguration.create(conf);
     decorateConf();
     this.metrics = new MetricsSink();
-    this.sharedHtableCon = ConnectionFactory.createConnection(this.conf);
   }
 
   /**
@@ -213,7 +212,9 @@ public class ReplicationSink {
    */
   public void stopReplicationSinkServices() {
     try {
-      this.sharedHtableCon.close();
+      if (this.sharedHtableCon != null) {
+        this.sharedHtableCon.close();
+      }
     } catch (IOException e) {
       LOG.warn("IOException while closing the connection", e); // ignoring as we are closing.
     }
@@ -232,6 +233,9 @@ public class ReplicationSink {
     }
     Table table = null;
     try {
+      if (this.sharedHtableCon == null) {
+        this.sharedHtableCon = ConnectionFactory.createConnection(this.conf);
+      }
       table = this.sharedHtableCon.getTable(tableName);
       for (List<Row> rows : allRows) {
         table.batch(rows);


Mime
View raw message