HBASE-14361 Investigate unused connection objects

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
chenheng 2015-09-09 15:40:22 +08:00 committed by stack
parent 945477e2e6
commit 6c16d244bb
1 changed files with 7 additions and 3 deletions

View File

@ -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);