Skip to content

Commit 531d290

Browse files
authored
HBASE-22612 Address the final overview reviewing comments of HBASE-21879 (#331)
1 parent c1e5350 commit 531d290

File tree

5 files changed

+25
-10
lines changed

5 files changed

+25
-10
lines changed

hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferListOutputStream.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@
4141
public class ByteBufferListOutputStream extends ByteBufferOutputStream {
4242
private static final Logger LOG = LoggerFactory.getLogger(ByteBufferListOutputStream.class);
4343

44-
private ByteBuffAllocator allocator;
44+
private final ByteBuffAllocator allocator;
4545
// Keep track of the BBs where bytes written to. We will first try to get a BB from the pool. If
4646
// it is not available will make a new one our own and keep writing to that. We keep track of all
4747
// the BBs that we got from pool, separately so that on closeAndPutbackBuffers, we can make sure

hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -193,8 +193,8 @@ public String getHFileName() {
193193
}
194194

195195
/**
196-
* HeapSize implementation. NOTE : The heapsize should be altered as and when new state variable
197-
* are added
196+
* HeapSize implementation. NOTE : The heap size should be altered when new state variable are
197+
* added.
198198
* @return heap size of the HFileContext
199199
*/
200200
@Override

hbase-common/src/main/java/org/apache/hadoop/hbase/nio/RefCnt.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@
2626

2727
/**
2828
* Maintain an reference count integer inside to track life cycle of {@link ByteBuff}, if the
29-
* reference count become 0, it'll call {@link Recycler#free()} once.
29+
* reference count become 0, it'll call {@link Recycler#free()} exactly once.
3030
*/
3131
@InterfaceAudience.Private
3232
public class RefCnt extends AbstractReferenceCounted {
@@ -36,8 +36,8 @@ public class RefCnt extends AbstractReferenceCounted {
3636
/**
3737
* Create an {@link RefCnt} with an initial reference count = 1. If the reference count become
3838
* zero, the recycler will do nothing. Usually, an Heap {@link ByteBuff} will use this kind of
39-
* refCnt to track its life cycle, it help to abstract the code path although it's meaningless to
40-
* use an refCnt for heap ByteBuff.
39+
* refCnt to track its life cycle, it help to abstract the code path although it's not really
40+
* needed to track on heap ByteBuff.
4141
*/
4242
public static RefCnt create() {
4343
return new RefCnt(ByteBuffAllocator.NONE);

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -153,8 +153,14 @@ public class LruBlockCache implements FirstLevelBlockCache {
153153
private static final String LRU_MAX_BLOCK_SIZE = "hbase.lru.max.block.size";
154154
private static final long DEFAULT_MAX_BLOCK_SIZE = 16L * 1024L * 1024L;
155155

156-
/** Concurrent map (the cache) */
157-
private transient final Map<BlockCacheKey, LruCachedBlock> map;
156+
/**
157+
* Defined the cache map as {@link ConcurrentHashMap} here, because in
158+
* {@link LruBlockCache#getBlock}, we need to guarantee the atomicity of map#computeIfPresent
159+
* (key, func). Besides, the func method must execute exactly once only when the key is present
160+
* and under the lock context, otherwise the reference count will be messed up. Notice that the
161+
* {@link java.util.concurrent.ConcurrentSkipListMap} can not guarantee that.
162+
*/
163+
private transient final ConcurrentHashMap<BlockCacheKey, LruCachedBlock> map;
158164

159165
/** Eviction lock (locked when eviction in process) */
160166
private transient final ReentrantLock evictionLock = new ReentrantLock(true);

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -140,7 +140,7 @@ public class BucketCache implements BlockCache, HeapSize {
140140
transient final RAMCache ramCache;
141141
// In this map, store the block's meta data like offset, length
142142
@VisibleForTesting
143-
transient ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
143+
transient ConcurrentHashMap<BlockCacheKey, BucketEntry> backingMap;
144144

145145
/**
146146
* Flag if the cache is enabled or not... We shut it off if there are IO
@@ -1526,7 +1526,16 @@ float getMemoryFactor() {
15261526
* Wrapped the delegate ConcurrentMap with maintaining its block's reference count.
15271527
*/
15281528
static class RAMCache {
1529-
final ConcurrentMap<BlockCacheKey, RAMQueueEntry> delegate = new ConcurrentHashMap<>();
1529+
/**
1530+
* Defined the map as {@link ConcurrentHashMap} explicitly here, because in
1531+
* {@link RAMCache#get(BlockCacheKey)} and
1532+
* {@link RAMCache#putIfAbsent(BlockCacheKey, RAMQueueEntry)} , we need to guarantee the
1533+
* atomicity of map#computeIfPresent(key, func) and map#putIfAbsent(key, func). Besides, the
1534+
* func method can execute exactly once only when the key is present(or absent) and under the
1535+
* lock context. Otherwise, the reference count of block will be messed up. Notice that the
1536+
* {@link java.util.concurrent.ConcurrentSkipListMap} can not guarantee that.
1537+
*/
1538+
final ConcurrentHashMap<BlockCacheKey, RAMQueueEntry> delegate = new ConcurrentHashMap<>();
15301539

15311540
public boolean containsKey(BlockCacheKey key) {
15321541
return delegate.containsKey(key);

0 commit comments

Comments
 (0)