HBASE-3673 Reduce HTable Pool Contention Using Concurrent Collections

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1085169 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-03-24 22:28:26 +00:00
parent 39bb359b53
commit a344cd98b7
2 changed files with 19 additions and 24 deletions

View File

@ -95,6 +95,8 @@ Release 0.91.0 - Unreleased
HBASE-3468 Enhance checkAndPut and checkAndDelete with comparators
HBASE-3683 NMapInputFormat should use a different config param for
number of maps
HBASE-3673 Reduce HTable Pool Contention Using Concurrent Collections
(Karthick Sankarachary via Stack)
TASK
HBASE-3559 Move report of split to master OFF the heartbeat channel

View File

@ -21,8 +21,10 @@ package org.apache.hadoop.hbase.client;
import java.util.LinkedList;
import java.util.Queue;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -42,8 +44,8 @@ import org.apache.hadoop.hbase.util.Bytes;
* <p>Pool will manage its own cluster to the cluster. See {@link HConnectionManager}.
*/
public class HTablePool {
private final ConcurrentMap<String, LinkedList<HTableInterface>> tables =
new ConcurrentHashMap<String, LinkedList<HTableInterface>>();
private final Map<String, Queue<HTableInterface>> tables =
new ConcurrentHashMap<String, Queue<HTableInterface>>();
private final Configuration config;
private final int maxSize;
private final HTableInterfaceFactory tableFactory;
@ -82,16 +84,13 @@ public class HTablePool {
* @throws RuntimeException if there is a problem instantiating the HTable
*/
public HTableInterface getTable(String tableName) {
LinkedList<HTableInterface> queue = tables.get(tableName);
Queue<HTableInterface> queue = tables.get(tableName);
if(queue == null) {
queue = new LinkedList<HTableInterface>();
tables.putIfAbsent(tableName, queue);
queue = new ConcurrentLinkedQueue<HTableInterface>();
tables.put(tableName, queue);
return createHTable(tableName);
}
HTableInterface table;
synchronized(queue) {
table = queue.poll();
}
HTableInterface table = queue.poll();
if(table == null) {
return createHTable(tableName);
}
@ -118,11 +117,9 @@ public class HTablePool {
* @param table table
*/
public void putTable(HTableInterface table) {
LinkedList<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
synchronized(queue) {
if(queue.size() >= maxSize) return;
queue.add(table);
}
Queue<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
if(queue.size() >= maxSize) return;
queue.add(table);
}
protected HTableInterface createHTable(String tableName) {
@ -140,12 +137,10 @@ public class HTablePool {
*/
public void closeTablePool(final String tableName) {
Queue<HTableInterface> queue = tables.get(tableName);
synchronized (queue) {
HTableInterface table = queue.poll();
while (table != null) {
this.tableFactory.releaseHTableInterface(table);
table = queue.poll();
}
HTableInterface table = queue.poll();
while (table != null) {
this.tableFactory.releaseHTableInterface(table);
table = queue.poll();
}
HConnectionManager.deleteConnection(this.config, true);
}
@ -161,8 +156,6 @@ public class HTablePool {
int getCurrentPoolSize(String tableName) {
Queue<HTableInterface> queue = tables.get(tableName);
synchronized(queue) {
return queue.size();
}
return queue.size();
}
}
}