ramkrish86 commented on a change in pull request #242: HBASE-22422 Retain an 
ByteBuff with refCnt=0 when getBlock from LRUCache
URL: https://github.com/apache/hbase/pull/242#discussion_r288391398
 
 

 ##########
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
 ##########
 @@ -1533,21 +1534,28 @@ public boolean containsKey(BlockCacheKey key) {
     }
 
     public RAMQueueEntry get(BlockCacheKey key) {
-      RAMQueueEntry re = delegate.get(key);
-      if (re != null) {
-        // It'll be referenced by RPC, so retain here.
+      return delegate.computeIfPresent(key, (k, re) -> {
+        // It'll be referenced by RPC, so retain atomically here. if the get 
and retain is not
+        // atomic, another thread may remove and release the block, when 
retaining in this thread we
+        // may retain a block with refCnt=0 which is disallowed. (see 
HBASE-22422)
         re.getData().retain();
-      }
-      return re;
+        return re;
+      });
     }
 
+    /**
+     * Return the previous associated value, or null if absent. It has the 
same meaning as
+     * {@link ConcurrentMap#putIfAbsent(Object, Object)}
+     */
     public RAMQueueEntry putIfAbsent(BlockCacheKey key, RAMQueueEntry entry) {
-      RAMQueueEntry previous = delegate.putIfAbsent(key, entry);
-      if (previous == null) {
+      AtomicBoolean absent = new AtomicBoolean(false);
+      RAMQueueEntry re = delegate.computeIfAbsent(key, k -> {
         // The RAMCache reference to this entry, so reference count should be 
increment.
         entry.getData().retain();
-      }
-      return previous;
+        absent.set(true);
+        return entry;
+      });
+      return absent.get() ? null : re;
     }
 
     public boolean remove(BlockCacheKey key) {
 
 Review comment:
   While removing the atomicity is not needed? Because we do computeIfAbsent 
and that is already now atomically guarded?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to