View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.io.hfile.slab;
20  
21  import java.nio.ByteBuffer;
22  import java.util.List;
23  import java.util.concurrent.ConcurrentMap;
24  import java.util.concurrent.atomic.AtomicLong;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.classification.InterfaceAudience;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.io.HeapSize;
31  import org.apache.hadoop.hbase.io.hfile.BlockCache;
32  import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
33  import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
34  import org.apache.hadoop.hbase.io.hfile.CacheStats;
35  import org.apache.hadoop.hbase.io.hfile.Cacheable;
36  import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.hbase.util.ClassSize;
39  import org.apache.hadoop.util.StringUtils;
40  
41  import com.google.common.cache.CacheBuilder;
42  import com.google.common.cache.RemovalListener;
43  import com.google.common.cache.RemovalNotification;
44  
45  /**
46   * SingleSizeCache is a slab allocated cache that caches elements up to a single
47   * size. It uses a slab allocator (Slab.java) to divide a direct bytebuffer,
48   * into evenly sized blocks. Any cached data will take up exactly 1 block. An
49   * exception will be thrown if the cached data cannot fit into the blockSize of
50   * this SingleSizeCache.
51   *
52   * Eviction and LRUness is taken care of by Guava's MapMaker, which creates a
53   * ConcurrentLinkedHashMap.
54   *
55   **/
56  @InterfaceAudience.Private
57  public class SingleSizeCache implements BlockCache, HeapSize {
58    private final Slab backingStore;
59    private final ConcurrentMap<BlockCacheKey, CacheablePair> backingMap;
60    private final int numBlocks;
61    private final int blockSize;
62    private final CacheStats stats;
63    private final SlabItemActionWatcher actionWatcher;
64    private final AtomicLong size;
65    private final AtomicLong timeSinceLastAccess;
66    public final static long CACHE_FIXED_OVERHEAD = ClassSize
67        .align((2 * Bytes.SIZEOF_INT) + (5 * ClassSize.REFERENCE)
68            + +ClassSize.OBJECT);
69  
70    static final Log LOG = LogFactory.getLog(SingleSizeCache.class);
71  
72    /**
73     * Default constructor. Specify the size of the blocks, number of blocks, and
74     * the SlabCache this cache will be assigned to.
75     *
76     *
77     * @param blockSize the size of each block, in bytes
78     *
79     * @param numBlocks the number of blocks of blockSize this cache will hold.
80     *
81     * @param master the SlabCache this SingleSlabCache is assigned to.
82     */
83    public SingleSizeCache(int blockSize, int numBlocks,
84        SlabItemActionWatcher master) {
85      this.blockSize = blockSize;
86      this.numBlocks = numBlocks;
87      backingStore = new Slab(blockSize, numBlocks);
88      this.stats = new CacheStats();
89      this.actionWatcher = master;
90      this.size = new AtomicLong(CACHE_FIXED_OVERHEAD + backingStore.heapSize());
91      this.timeSinceLastAccess = new AtomicLong();
92  
93      // This evictionListener is called whenever the cache automatically
94      // evicts
95      // something.
96      RemovalListener<BlockCacheKey, CacheablePair> listener =
97        new RemovalListener<BlockCacheKey, CacheablePair>() {
98          @Override
99          public void onRemoval(
100             RemovalNotification<BlockCacheKey, CacheablePair> notification) {
101           if (!notification.wasEvicted()) {
102             // Only process removals by eviction, not by replacement or
103             // explicit removal
104             return;
105           }
106           CacheablePair value = notification.getValue();
107           timeSinceLastAccess.set(System.nanoTime()
108               - value.recentlyAccessed.get());
109           stats.evict();
110           doEviction(notification.getKey(), value);
111         }
112       };
113 
114     backingMap = CacheBuilder.newBuilder()
115         .maximumSize(numBlocks - 1)
116         .removalListener(listener)
117         .<BlockCacheKey, CacheablePair>build()
118         .asMap();
119 
120 
121   }
122 
123   @Override
124   public void cacheBlock(BlockCacheKey blockName, Cacheable toBeCached) {
125     ByteBuffer storedBlock;
126 
127     try {
128       storedBlock = backingStore.alloc(toBeCached.getSerializedLength());
129     } catch (InterruptedException e) {
130       LOG.warn("SlabAllocator was interrupted while waiting for block to become available");
131       LOG.warn(e);
132       return;
133     }
134 
135     CacheablePair newEntry = new CacheablePair(toBeCached.getDeserializer(),
136         storedBlock);
137     toBeCached.serialize(storedBlock);
138 
139     synchronized (this) {
140       CacheablePair alreadyCached = backingMap.putIfAbsent(blockName, newEntry);
141     
142 
143       if (alreadyCached != null) {
144         backingStore.free(storedBlock);
145         throw new RuntimeException("already cached " + blockName);
146       }
147       if (actionWatcher != null) {
148         actionWatcher.onInsertion(blockName, this);
149       }
150     }
151     newEntry.recentlyAccessed.set(System.nanoTime());
152     this.size.addAndGet(newEntry.heapSize());
153   }
154 
155   @Override
156   public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat) {
157     CacheablePair contentBlock = backingMap.get(key);
158     if (contentBlock == null) {
159       if (!repeat) stats.miss(caching);
160       return null;
161     }
162 
163     stats.hit(caching);
164     // If lock cannot be obtained, that means we're undergoing eviction.
165     try {
166       contentBlock.recentlyAccessed.set(System.nanoTime());
167       synchronized (contentBlock) {
168         if (contentBlock.serializedData == null) {
169           // concurrently evicted
170           LOG.warn("Concurrent eviction of " + key);
171           return null;
172         }
173         return contentBlock.deserializer
174             .deserialize(contentBlock.serializedData.asReadOnlyBuffer());
175       }
176     } catch (Throwable t) {
177       LOG.error("Deserializer threw an exception. This may indicate a bug.", t);
178       return null;
179     }
180   }
181 
182   /**
183    * Evicts the block
184    *
185    * @param key the key of the entry we are going to evict
186    * @return the evicted ByteBuffer
187    */
188   public boolean evictBlock(BlockCacheKey key) {
189     stats.evict();
190     CacheablePair evictedBlock = backingMap.remove(key);
191 
192     if (evictedBlock != null) {
193       doEviction(key, evictedBlock);
194     }
195     return evictedBlock != null;
196 
197   }
198 
199   private void doEviction(BlockCacheKey key, CacheablePair evictedBlock) {
200     long evictedHeap = 0;
201     synchronized (evictedBlock) {
202       if (evictedBlock.serializedData == null) {
203         // someone else already freed
204         return;
205       }
206       evictedHeap = evictedBlock.heapSize();
207       ByteBuffer bb = evictedBlock.serializedData;
208       evictedBlock.serializedData = null;
209       backingStore.free(bb);
210 
211       // We have to do this callback inside the synchronization here.
212       // Otherwise we can have the following interleaving:
213       // Thread A calls getBlock():
214       // SlabCache directs call to this SingleSizeCache
215       // It gets the CacheablePair object
216       // Thread B runs eviction
217       // doEviction() is called and sets serializedData = null, here.
218       // Thread A sees the null serializedData, and returns null
219       // Thread A calls cacheBlock on the same block, and gets
220       // "already cached" since the block is still in backingStore
221 
222       if (actionWatcher != null) {
223         actionWatcher.onEviction(key, this);
224       }
225     }
226     stats.evicted();
227     size.addAndGet(-1 * evictedHeap);
228   }
229 
230   public void logStats() {
231 
232     long milliseconds = this.timeSinceLastAccess.get() / 1000000;
233 
234     LOG.info("For Slab of size " + this.blockSize + ": "
235         + this.getOccupiedSize() / this.blockSize
236         + " occupied, out of a capacity of " + this.numBlocks
237         + " blocks. HeapSize is "
238         + StringUtils.humanReadableInt(this.heapSize()) + " bytes." + ", "
239         + "churnTime=" + StringUtils.formatTime(milliseconds));
240 
241     LOG.info("Slab Stats: " + "accesses="
242         + stats.getRequestCount()
243         + ", "
244         + "hits="
245         + stats.getHitCount()
246         + ", "
247         + "hitRatio="
248         + (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(
249             stats.getHitRatio(), 2) + "%, "))
250         + "cachingAccesses="
251         + stats.getRequestCachingCount()
252         + ", "
253         + "cachingHits="
254         + stats.getHitCachingCount()
255         + ", "
256         + "cachingHitsRatio="
257         + (stats.getHitCachingCount() == 0 ? "0" : (StringUtils.formatPercent(
258             stats.getHitCachingRatio(), 2) + "%, ")) + "evictions="
259         + stats.getEvictionCount() + ", " + "evicted="
260         + stats.getEvictedCount() + ", " + "evictedPerRun="
261         + stats.evictedPerEviction());
262 
263   }
264 
265   public void shutdown() {
266     backingStore.shutdown();
267   }
268 
269   public long heapSize() {
270     return this.size.get() + backingStore.heapSize();
271   }
272 
273   public long size() {
274     return (long) this.blockSize * (long) this.numBlocks;
275   }
276 
277   public long getFreeSize() {
278     return (long) backingStore.getBlocksRemaining() * (long) blockSize;
279   }
280 
281   public long getOccupiedSize() {
282     return (long) (numBlocks - backingStore.getBlocksRemaining()) * (long) blockSize;
283   }
284 
285   public long getEvictedCount() {
286     return stats.getEvictedCount();
287   }
288 
289   public CacheStats getStats() {
290     return this.stats;
291   }
292 
293   @Override
294   public long getBlockCount() {
295     return numBlocks - backingStore.getBlocksRemaining();
296   }
297 
298   /* Since its offheap, it doesn't matter if its in memory or not */
299   @Override
300   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
301     this.cacheBlock(cacheKey, buf);
302   }
303 
304   /*
305    * This is never called, as evictions are handled in the SlabCache layer,
306    * implemented in the event we want to use this as a standalone cache.
307    */
308   @Override
309   public int evictBlocksByHfileName(String hfileName) {
310     int evictedCount = 0;
311     for (BlockCacheKey e : backingMap.keySet()) {
312       if (e.getHfileName().equals(hfileName)) {
313         this.evictBlock(e);
314       }
315     }
316     return evictedCount;
317   }
318 
319   @Override
320   public long getCurrentSize() {
321     return 0;
322   }
323 
324   /*
325    * Not implemented. Extremely costly to do this from the off heap cache, you'd
326    * need to copy every object on heap once
327    */
328   @Override
329   public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
330       Configuration conf) {
331     throw new UnsupportedOperationException();
332   }
333 
334   /* Just a pair class, holds a reference to the parent cacheable */
335   private static class CacheablePair implements HeapSize {
336     final CacheableDeserializer<Cacheable> deserializer;
337     ByteBuffer serializedData;
338     AtomicLong recentlyAccessed;
339 
340     private CacheablePair(CacheableDeserializer<Cacheable> deserializer,
341         ByteBuffer serializedData) {
342       this.recentlyAccessed = new AtomicLong();
343       this.deserializer = deserializer;
344       this.serializedData = serializedData;
345     }
346 
347     /*
348      * Heapsize overhead of this is the default object overhead, the heapsize of
349      * the serialized object, and the cost of a reference to the bytebuffer,
350      * which is already accounted for in SingleSizeCache
351      */
352     @Override
353     public long heapSize() {
354       return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE * 3
355           + ClassSize.ATOMIC_LONG);
356     }
357   }
358 }