[GitHub] [hbase] virajjasani commented on a change in pull request #3407: HBASE-26018 Perf improvement in L1 cache - Optimistic call to buffer.retain()

2021-08-12 Thread GitBox


virajjasani commented on a change in pull request #3407:
URL: https://github.com/apache/hbase/pull/3407#discussion_r687501045



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java
##
@@ -646,14 +639,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean 
evict) {
   @Override
   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean 
repeat,
 boolean updateCacheMetrics) {
-LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -> {
-  // It will be referenced by RPC path, so increase here. NOTICE: Must do 
the retain inside
-  // this block. because if retain outside the map#computeIfPresent, the 
evictBlock may remove
-  // the block and release, then we're retaining a block with refCnt=0 
which is disallowed.
-  // see HBASE-22422.
-  val.getBuffer().retain();
-  return val;
-});
+LruCachedBlock cb = map.get(cacheKey);
+if (cb != null) {
+  try {
+cb.getBuffer().retain();

Review comment:
   > A cached buffer item detached from the cache still needs to have its 
#release called w/ refcount at zero so the backing memory gets readded to the 
pool.
   
   Yeah I think this makes sense. Let me get back to this in case I find some 
better and obvious way to improve perf and get some YCSB results.




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

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] virajjasani commented on a change in pull request #3407: HBASE-26018 Perf improvement in L1 cache - Optimistic call to buffer.retain()

2021-07-17 Thread GitBox


virajjasani commented on a change in pull request #3407:
URL: https://github.com/apache/hbase/pull/3407#discussion_r671717888



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java
##
@@ -646,14 +639,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean 
evict) {
   @Override
   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean 
repeat,
 boolean updateCacheMetrics) {
-LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -> {
-  // It will be referenced by RPC path, so increase here. NOTICE: Must do 
the retain inside
-  // this block. because if retain outside the map#computeIfPresent, the 
evictBlock may remove
-  // the block and release, then we're retaining a block with refCnt=0 
which is disallowed.
-  // see HBASE-22422.
-  val.getBuffer().retain();
-  return val;
-});
+LruCachedBlock cb = map.get(cacheKey);
+if (cb != null) {
+  try {
+cb.getBuffer().retain();

Review comment:
   > Does the CB get returned to the cache when done?
   
   You mean if CB gets returned to L1 cache (CHM) after it's buffer has served 
read request? Yes, that's the case (unless I misunderstood the question)




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

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] virajjasani commented on a change in pull request #3407: HBASE-26018 Perf improvement in L1 cache - Optimistic call to buffer.retain()

2021-07-09 Thread GitBox


virajjasani commented on a change in pull request #3407:
URL: https://github.com/apache/hbase/pull/3407#discussion_r666785190



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java
##
@@ -646,14 +639,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean 
evict) {
   @Override
   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean 
repeat,
 boolean updateCacheMetrics) {
-LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -> {
-  // It will be referenced by RPC path, so increase here. NOTICE: Must do 
the retain inside
-  // this block. because if retain outside the map#computeIfPresent, the 
evictBlock may remove
-  // the block and release, then we're retaining a block with refCnt=0 
which is disallowed.
-  // see HBASE-22422.
-  val.getBuffer().retain();
-  return val;
-});
+LruCachedBlock cb = map.get(cacheKey);
+if (cb != null) {
+  try {
+cb.getBuffer().retain();

Review comment:
   Thanks @saintstack.
   
   > After, biggest locking consumer was elsewhere and much more insignificant 
percentage
   
   Does this mean we can kind of ignore this case (assuming objects not in 
cache will get GC'ed regardless of their netty based refCount)? Still thinking 
about this.




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

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] virajjasani commented on a change in pull request #3407: HBASE-26018 Perf improvement in L1 cache - Optimistic call to buffer.retain()

2021-07-08 Thread GitBox


virajjasani commented on a change in pull request #3407:
URL: https://github.com/apache/hbase/pull/3407#discussion_r666482218



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java
##
@@ -646,14 +639,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean 
evict) {
   @Override
   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean 
repeat,
 boolean updateCacheMetrics) {
-LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -> {
-  // It will be referenced by RPC path, so increase here. NOTICE: Must do 
the retain inside
-  // this block. because if retain outside the map#computeIfPresent, the 
evictBlock may remove
-  // the block and release, then we're retaining a block with refCnt=0 
which is disallowed.
-  // see HBASE-22422.
-  val.getBuffer().retain();
-  return val;
-});
+LruCachedBlock cb = map.get(cacheKey);
+if (cb != null) {
+  try {
+cb.getBuffer().retain();

Review comment:
   I think for possibility#2 in above, we stand a chance where buffer with 
non-zero refCount is not in the cache. I see, let me see what alternatives we 
have for this case. 
   Although I still think that same case can happen even today.
   getBlock does retain() which will bring refCount of BB to 2, while getBlock 
is busy updating stats, eviction thread can evict block from cache and it does 
release() which will bring refCount of BB to 1. So even in this case, we can 
positive refCount buffer which is evicted from cache.




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

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] virajjasani commented on a change in pull request #3407: HBASE-26018 Perf improvement in L1 cache - Optimistic call to buffer.retain()

2021-07-08 Thread GitBox


virajjasani commented on a change in pull request #3407:
URL: https://github.com/apache/hbase/pull/3407#discussion_r666482218



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java
##
@@ -646,14 +639,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean 
evict) {
   @Override
   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean 
repeat,
 boolean updateCacheMetrics) {
-LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -> {
-  // It will be referenced by RPC path, so increase here. NOTICE: Must do 
the retain inside
-  // this block. because if retain outside the map#computeIfPresent, the 
evictBlock may remove
-  // the block and release, then we're retaining a block with refCnt=0 
which is disallowed.
-  // see HBASE-22422.
-  val.getBuffer().retain();
-  return val;
-});
+LruCachedBlock cb = map.get(cacheKey);
+if (cb != null) {
+  try {
+cb.getBuffer().retain();

Review comment:
   I think for possibility#2 in above, we stand a chance where buffer with 
non-zero refCount is not in the cache. I see, let me see what alternatives we 
have for this case.




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

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] virajjasani commented on a change in pull request #3407: HBASE-26018 Perf improvement in L1 cache - Optimistic call to buffer.retain()

2021-07-08 Thread GitBox


virajjasani commented on a change in pull request #3407:
URL: https://github.com/apache/hbase/pull/3407#discussion_r666473427



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java
##
@@ -646,14 +639,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean 
evict) {
   @Override
   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean 
repeat,
 boolean updateCacheMetrics) {
-LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -> {
-  // It will be referenced by RPC path, so increase here. NOTICE: Must do 
the retain inside
-  // this block. because if retain outside the map#computeIfPresent, the 
evictBlock may remove
-  // the block and release, then we're retaining a block with refCnt=0 
which is disallowed.
-  // see HBASE-22422.
-  val.getBuffer().retain();
-  return val;
-});
+LruCachedBlock cb = map.get(cacheKey);
+if (cb != null) {
+  try {
+cb.getBuffer().retain();

Review comment:
   It is purged from cache by:
   ```
 protected long evictBlock(LruCachedBlock block, boolean 
evictedByEvictionProcess) {
   LruCachedBlock previous = map.remove(block.getCacheKey());===> 
removed from map
   if (previous == null) {
 return 0;
   }
   updateSizeMetrics(block, true);
   long val = elements.decrementAndGet();
   if (LOG.isTraceEnabled()) {
 long size = map.size();
 assertCounterSanity(size, val);
   }
   if (block.getBuffer().getBlockType().isData()) {
 dataBlockElements.decrement();
   }
   if (evictedByEvictionProcess) {
 // When the eviction of the block happened because of invalidation of 
HFiles, no need to
 // update the stats counter.
 stats.evicted(block.getCachedTime(), block.getCacheKey().isPrimary());
 if (victimHandler != null) {
   victimHandler.cacheBlock(block.getCacheKey(), block.getBuffer());
 }
   }
   // Decrease the block's reference count, and if refCount is 0, then 
it'll auto-deallocate. DO
   // NOT move this up because if do that then the victimHandler may access 
the buffer with
   // refCnt = 0 which is disallowed.
   previous.getBuffer().release(); > 
buffer released
   return block.heapSize();
 }
   ```
   
   Based on above mentioned eviction code, we have below mentioned 
possibilities when eviction and getBlock happens for the same block at the same 
time:
   
   1. getBlock retrieves block from map, eviction removes it from map, eviction 
does release(), getBlock does retain() and encounters IllegalRefCount 
Exception, we handler it with this patch and treat it as cache miss.
   2. getBlock retrieves block from map, eviction removes it from map, getBlock 
does retain(), eviction does release(). Since getBlock retain() was successful, 
it proceeds as successful cache hit, which happens even today with 
computeIfPresent. Subsequent getBlock call will return null as block was 
evicted previously.
   3. eviction removes from map, getBlock gets null, it's clear cache miss.
   
   I think we seem good here. WDYT @saintstack?




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

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org