Author: stack
Date: Wed Sep 8 22:41:27 2010
New Revision: 995273
URL: http://svn.apache.org/viewvc?rev=995273&view=rev
Log:
HBASE-2969 missing sync in HTablePool.getTable()
Modified:
hbase/trunk/CHANGES.txt
hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
Modified: hbase/trunk/CHANGES.txt
URL:
http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=995273&r1=995272&r2=995273&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed Sep 8 22:41:27 2010
@@ -511,6 +511,8 @@ Release 0.21.0 - Unreleased
versions of a column (pranav via jgray)
HBASE-2967 Failed split: IOE 'File is Corrupt!' -- sync length not being
written out to SequenceFile
+ HBASE-2969 missing sync in HTablePool.getTable()
+ (Guilherme Mauro Germoglio Barbosa via Stack)
IMPROVEMENTS
HBASE-1760 Cleanup TODOs in HTable
Modified:
hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
URL:
http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=995273&r1=995272&r2=995273&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
(original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
Wed Sep 8 22:41:27 2010
@@ -23,6 +23,8 @@ import org.apache.hadoop.conf.Configurat
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.util.Bytes;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
@@ -41,8 +43,8 @@ import java.util.Queue;
* is {...@link Integer#MAX_VALUE}.<p>
*/
public class HTablePool {
- private final Map<String, LinkedList<HTableInterface>> tables =
- Collections.synchronizedMap(new HashMap<String,
LinkedList<HTableInterface>>());
+ private final ConcurrentMap<String, LinkedList<HTableInterface>> tables =
+ new ConcurrentHashMap<String, LinkedList<HTableInterface>>();
private final Configuration config;
private final int maxSize;
private HTableInterfaceFactory tableFactory = new HTableFactory();
@@ -82,7 +84,7 @@ public class HTablePool {
LinkedList<HTableInterface> queue = tables.get(tableName);
if(queue == null) {
queue = new LinkedList<HTableInterface>();
- tables.put(tableName, queue);
+ tables.putIfAbsent(tableName, queue);
return createHTable(tableName);
}
HTableInterface table;