commit a1cf21deb992c927184be258432624c53a4225dd Author: Karthick Sankarachary Date: Fri Mar 18 15:30:12 2011 -0700 HBASE-3673 Reduce HTable Pool Contention Using Concurrent Collections diff --git a/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java b/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java index 0a7bc4f..f1bb59c 100755 --- a/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java +++ b/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java @@ -28,6 +28,8 @@ import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; /** * A simple pool of HTable instances.

@@ -41,8 +43,8 @@ import java.util.Queue; * is {@link Integer#MAX_VALUE}.

*/ public class HTablePool { - private final Map> tables = - Collections.synchronizedMap(new HashMap>()); + private final Map> tables = + new ConcurrentHashMap>(); private final Configuration config; private final int maxSize; private HTableInterfaceFactory tableFactory = new HTableFactory(); @@ -79,16 +81,13 @@ public class HTablePool { * @throws RuntimeException if there is a problem instantiating the HTable */ public HTableInterface getTable(String tableName) { - LinkedList queue = tables.get(tableName); + Queue queue = tables.get(tableName); if(queue == null) { - queue = new LinkedList(); + queue = new ConcurrentLinkedQueue(); tables.put(tableName, queue); return createHTable(tableName); } - HTableInterface table; - synchronized(queue) { - table = queue.poll(); - } + HTableInterface table = queue.poll(); if(table == null) { return createHTable(tableName); } @@ -115,11 +114,9 @@ public class HTablePool { * @param table table */ public void putTable(HTableInterface table) { - LinkedList queue = tables.get(Bytes.toString(table.getTableName())); - synchronized(queue) { - if(queue.size() >= maxSize) return; - queue.add(table); - } + Queue queue = tables.get(Bytes.toString(table.getTableName())); + if(queue.size() >= maxSize) return; + queue.add(table); } protected HTableInterface createHTable(String tableName) { @@ -137,14 +134,11 @@ public class HTablePool { */ public void closeTablePool(final String tableName) { Queue 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(); } - } /** @@ -158,8 +152,6 @@ public class HTablePool { int getCurrentPoolSize(String tableName) { Queue queue = tables.get(tableName); - synchronized(queue) { - return queue.size(); - } + return queue.size(); } }