|
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
@@ -29,8 +28,8 @@ import java.util.Random;
|
|
|
import java.util.SortedMap;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.TreeSet;
|
|
|
-import java.util.Vector;
|
|
|
import java.util.Map.Entry;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
@@ -142,13 +141,13 @@ public class HRegion implements HConstants {
|
|
|
LOG.info("starting merge of regions: " + a.getRegionName() + " and " +
|
|
|
b.getRegionName() + " into new region " + newRegionInfo.toString());
|
|
|
|
|
|
- Map<Text, Vector<HStoreFile>> byFamily =
|
|
|
- new TreeMap<Text, Vector<HStoreFile>>();
|
|
|
+ Map<Text, List<HStoreFile>> byFamily =
|
|
|
+ new TreeMap<Text, List<HStoreFile>>();
|
|
|
byFamily = filesByFamily(byFamily, a.close());
|
|
|
byFamily = filesByFamily(byFamily, b.close());
|
|
|
- for (Map.Entry<Text, Vector<HStoreFile>> es : byFamily.entrySet()) {
|
|
|
+ for (Map.Entry<Text, List<HStoreFile>> es : byFamily.entrySet()) {
|
|
|
Text colFamily = es.getKey();
|
|
|
- Vector<HStoreFile> srcFiles = es.getValue();
|
|
|
+ List<HStoreFile> srcFiles = es.getValue();
|
|
|
HStoreFile dst = new HStoreFile(conf, merges,
|
|
|
HRegionInfo.encodeRegionName(newRegionInfo.getRegionName()),
|
|
|
colFamily, Math.abs(rand.nextLong()));
|
|
@@ -175,12 +174,12 @@ public class HRegion implements HConstants {
|
|
|
* @param storeFiles Store files to process.
|
|
|
* @return Returns <code>byFamily</code>
|
|
|
*/
|
|
|
- private static Map<Text, Vector<HStoreFile>> filesByFamily(
|
|
|
- Map<Text, Vector<HStoreFile>> byFamily, Vector<HStoreFile> storeFiles) {
|
|
|
+ private static Map<Text, List<HStoreFile>> filesByFamily(
|
|
|
+ Map<Text, List<HStoreFile>> byFamily, List<HStoreFile> storeFiles) {
|
|
|
for(HStoreFile src: storeFiles) {
|
|
|
- Vector<HStoreFile> v = byFamily.get(src.getColFamily());
|
|
|
+ List<HStoreFile> v = byFamily.get(src.getColFamily());
|
|
|
if(v == null) {
|
|
|
- v = new Vector<HStoreFile>();
|
|
|
+ v = new ArrayList<HStoreFile>();
|
|
|
byFamily.put(src.getColFamily(), v);
|
|
|
}
|
|
|
v.add(src);
|
|
@@ -192,11 +191,11 @@ public class HRegion implements HConstants {
|
|
|
// Members
|
|
|
//////////////////////////////////////////////////////////////////////////////
|
|
|
|
|
|
- volatile Map<Text, Long> rowsToLocks = new HashMap<Text, Long>();
|
|
|
- volatile Map<Long, Text> locksToRows = new HashMap<Long, Text>();
|
|
|
- volatile Map<Text, HStore> stores = new HashMap<Text, HStore>();
|
|
|
+ volatile Map<Text, Long> rowsToLocks = new ConcurrentHashMap<Text, Long>();
|
|
|
+ volatile Map<Long, Text> locksToRows = new ConcurrentHashMap<Long, Text>();
|
|
|
+ volatile Map<Text, HStore> stores = new ConcurrentHashMap<Text, HStore>();
|
|
|
volatile Map<Long, TreeMap<HStoreKey, byte []>> targetColumns =
|
|
|
- new HashMap<Long, TreeMap<HStoreKey, byte []>>();
|
|
|
+ new ConcurrentHashMap<Long, TreeMap<HStoreKey, byte []>>();
|
|
|
|
|
|
final AtomicLong memcacheSize = new AtomicLong(0);
|
|
|
|
|
@@ -359,7 +358,7 @@ public class HRegion implements HConstants {
|
|
|
*
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public Vector<HStoreFile> close() throws IOException {
|
|
|
+ public List<HStoreFile> close() throws IOException {
|
|
|
return close(false);
|
|
|
}
|
|
|
|
|
@@ -377,7 +376,7 @@ public class HRegion implements HConstants {
|
|
|
*
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- Vector<HStoreFile> close(boolean abort) throws IOException {
|
|
|
+ List<HStoreFile> close(boolean abort) throws IOException {
|
|
|
if (isClosed()) {
|
|
|
LOG.info("region " + this.regionInfo.getRegionName() + " already closed");
|
|
|
return null;
|
|
@@ -421,7 +420,7 @@ public class HRegion implements HConstants {
|
|
|
internalFlushcache(snapshotMemcaches());
|
|
|
}
|
|
|
|
|
|
- Vector<HStoreFile> result = new Vector<HStoreFile>();
|
|
|
+ List<HStoreFile> result = new ArrayList<HStoreFile>();
|
|
|
for (HStore store: stores.values()) {
|
|
|
result.addAll(store.close());
|
|
|
}
|
|
@@ -571,7 +570,7 @@ public class HRegion implements HConstants {
|
|
|
// Now close the HRegion. Close returns all store files or null if not
|
|
|
// supposed to close (? What to do in this case? Implement abort of close?)
|
|
|
// Close also does wait on outstanding rows and calls a flush just-in-case.
|
|
|
- Vector<HStoreFile> hstoreFilesToSplit = close();
|
|
|
+ List<HStoreFile> hstoreFilesToSplit = close();
|
|
|
if (hstoreFilesToSplit == null) {
|
|
|
LOG.warn("Close came back null (Implement abort of close?)");
|
|
|
throw new RuntimeException("close returned empty vector of HStoreFiles");
|
|
@@ -909,6 +908,7 @@ public class HRegion implements HConstants {
|
|
|
// A. Flush memcache to all the HStores.
|
|
|
// Keep running vector of all store files that includes both old and the
|
|
|
// just-made new flush store file.
|
|
|
+
|
|
|
for (HStore hstore: stores.values()) {
|
|
|
hstore.flushCache(sequenceId);
|
|
|
}
|