private void doEviction(BlockCacheKey key, CacheablePair evictedBlock) { long evictedHeap = 0; synchronized (evictedBlock) { if (evictedBlock.serializedData == null) { // someone else already freed return; } evictedHeap = evictedBlock.heapSize(); ByteBuffer bb = evictedBlock.serializedData; evictedBlock.serializedData = null; backingStore.free(bb); // We have to do this callback inside the synchronization here. // Otherwise we can have the following interleaving: // Thread A calls getBlock(): // SlabCache directs call to this SingleSizeCache // It gets the CacheablePair object // Thread B runs eviction // doEviction() is called and sets serializedData = null, here. // Thread A sees the null serializedData, and returns null // Thread A calls cacheBlock on the same block, and gets // "already cached" since the block is still in backingStore if (actionWatcher != null) { actionWatcher.onEviction(key, this); } } stats.evicted(); size.addAndGet(-1 * evictedHeap); }
@Override public void cacheBlock(BlockCacheKey blockName, Cacheable toBeCached) { ByteBuffer storedBlock; try { storedBlock = backingStore.alloc(toBeCached.getSerializedLength()); } catch (InterruptedException e) { LOG.warn("SlabAllocator was interrupted while waiting for block to become available"); LOG.warn(e); return; } CacheablePair newEntry = new CacheablePair(toBeCached.getDeserializer(), storedBlock); toBeCached.serialize(storedBlock); synchronized (this) { CacheablePair alreadyCached = backingMap.putIfAbsent(blockName, newEntry); if (alreadyCached != null) { backingStore.free(storedBlock); throw new RuntimeException("already cached " + blockName); } if (actionWatcher != null) { actionWatcher.onInsertion(blockName, this); } } newEntry.recentlyAccessed.set(System.nanoTime()); this.size.addAndGet(newEntry.heapSize()); }