HBASE-2969 missing sync in HTablePool.getTable()

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@995273 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-09-08 22:41:27 +00:00
parent cf43037b3b
commit e144e8f32d
2 changed files with 7 additions and 3 deletions

View File

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

View File

@ -23,6 +23,8 @@ import org.apache.hadoop.conf.Configuration;
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;