View Javadoc

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