From 349a26a434ebaf853bb82f2c63118a0e17813983 Mon Sep 17 00:00:00 2001 From: chenheng Date: Wed, 9 Sep 2015 15:40:22 +0800 Subject: [PATCH] HBASE-14361 Investigate unused connection objects --- .../hbase/replication/regionserver/ReplicationSink.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) 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 fc7aa8e..4869545 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,9 +72,10 @@ 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(); + private final Object sharedHtableConLock = new Object(); /** * Create a sink for replication @@ -88,7 +89,6 @@ public class ReplicationSink { this.conf = HBaseConfiguration.create(conf); decorateConf(); this.metrics = new MetricsSink(); - this.sharedHtableCon = ConnectionFactory.createConnection(this.conf); } /** @@ -213,7 +213,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 +234,13 @@ public class ReplicationSink { } Table table = null; try { + if (this.sharedHtableCon == null) { + synchronized (sharedHtableConLock) { + if (this.sharedHtableCon == null) { + this.sharedHtableCon = ConnectionFactory.createConnection(this.conf); + } + } + } table = this.sharedHtableCon.getTable(tableName); for (List rows : allRows) { table.batch(rows, null); -- 1.9.3 (Apple Git-50)