View Javadoc

1   /**
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one or more
5    * contributor license agreements. See the NOTICE file distributed with this
6    * work for additional information regarding copyright ownership. The ASF
7    * licenses this file to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance with the License.
9    * 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, WITHOUT
15   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
16   * License for the specific language governing permissions and limitations
17   * under the License.
18   */
19  package org.apache.hadoop.hbase.io.hfile;
20  
21  import java.util.Iterator;
22  
23  import org.apache.hadoop.hbase.classification.InterfaceAudience;
24  import org.apache.hadoop.hbase.io.HeapSize;
25  import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
26  import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
27  
28  
29  /**
30   * CombinedBlockCache is an abstraction layer that combines
31   * {@link LruBlockCache} and {@link BucketCache}. The smaller lruCache is used
32   * to cache bloom blocks and index blocks.  The larger l2Cache is used to
33   * cache data blocks. {@link #getBlock(BlockCacheKey, boolean, boolean, boolean)} reads
34   * first from the smaller lruCache before looking for the block in the l2Cache.  Blocks evicted
35   * from lruCache are put into the bucket cache. 
36   * Metrics are the combined size and hits and misses of both caches.
37   * 
38   */
39  @InterfaceAudience.Private
40  public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
41    protected final LruBlockCache lruCache;
42    protected final BlockCache l2Cache;
43    protected final CombinedCacheStats combinedCacheStats;
44  
45    public CombinedBlockCache(LruBlockCache lruCache, BlockCache l2Cache) {
46      this.lruCache = lruCache;
47      this.l2Cache = l2Cache;
48      this.combinedCacheStats = new CombinedCacheStats(lruCache.getStats(),
49          l2Cache.getStats());
50    }
51  
52    @Override
53    public long heapSize() {
54      long l2size = 0;
55      if (l2Cache instanceof HeapSize) {
56        l2size = ((HeapSize) l2Cache).heapSize();
57      }
58      return lruCache.heapSize() + l2size;
59    }
60  
61    @Override
62    public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
63        final boolean cacheDataInL1) {
64      boolean isMetaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
65      if (isMetaBlock || cacheDataInL1) {
66        lruCache.cacheBlock(cacheKey, buf, inMemory, cacheDataInL1);
67      } else {
68        l2Cache.cacheBlock(cacheKey, buf, inMemory, false);
69      }
70    }
71  
72    @Override
73    public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
74      cacheBlock(cacheKey, buf, false, false);
75    }
76  
77    @Override
78    public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching,
79        boolean repeat, boolean updateCacheMetrics) {
80      // TODO: is there a hole here, or just awkwardness since in the lruCache getBlock
81      // we end up calling l2Cache.getBlock.
82      if (lruCache.containsBlock(cacheKey)) {
83        return lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
84      }
85      Cacheable result = l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
86  
87      return result;
88    }
89  
90    @Override
91    public boolean evictBlock(BlockCacheKey cacheKey) {
92      return lruCache.evictBlock(cacheKey) || l2Cache.evictBlock(cacheKey);
93    }
94  
95    @Override
96    public int evictBlocksByHfileName(String hfileName) {
97      return lruCache.evictBlocksByHfileName(hfileName)
98          + l2Cache.evictBlocksByHfileName(hfileName);
99    }
100 
101   @Override
102   public CacheStats getStats() {
103     return this.combinedCacheStats;
104   }
105 
106   @Override
107   public void shutdown() {
108     lruCache.shutdown();
109     l2Cache.shutdown();
110   }
111 
112   @Override
113   public long size() {
114     return lruCache.size() + l2Cache.size();
115   }
116 
117   @Override
118   public long getFreeSize() {
119     return lruCache.getFreeSize() + l2Cache.getFreeSize();
120   }
121 
122   @Override
123   public long getCurrentSize() {
124     return lruCache.getCurrentSize() + l2Cache.getCurrentSize();
125   }
126 
127   @Override
128   public long getBlockCount() {
129     return lruCache.getBlockCount() + l2Cache.getBlockCount();
130   }
131 
132   public static class CombinedCacheStats extends CacheStats {
133     private final CacheStats lruCacheStats;
134     private final CacheStats bucketCacheStats;
135 
136     CombinedCacheStats(CacheStats lbcStats, CacheStats fcStats) {
137       super("CombinedBlockCache");
138       this.lruCacheStats = lbcStats;
139       this.bucketCacheStats = fcStats;
140     }
141 
142     @Override
143     public long getRequestCount() {
144       return lruCacheStats.getRequestCount()
145           + bucketCacheStats.getRequestCount();
146     }
147 
148     @Override
149     public long getRequestCachingCount() {
150       return lruCacheStats.getRequestCachingCount()
151           + bucketCacheStats.getRequestCachingCount();
152     }
153 
154     @Override
155     public long getMissCount() {
156       return lruCacheStats.getMissCount() + bucketCacheStats.getMissCount();
157     }
158 
159     @Override
160     public long getMissCachingCount() {
161       return lruCacheStats.getMissCachingCount()
162           + bucketCacheStats.getMissCachingCount();
163     }
164 
165     @Override
166     public long getHitCount() {
167       return lruCacheStats.getHitCount() + bucketCacheStats.getHitCount();
168     }
169 
170     @Override
171     public long getHitCachingCount() {
172       return lruCacheStats.getHitCachingCount()
173           + bucketCacheStats.getHitCachingCount();
174     }
175 
176     @Override
177     public long getEvictionCount() {
178       return lruCacheStats.getEvictionCount()
179           + bucketCacheStats.getEvictionCount();
180     }
181 
182     @Override
183     public long getEvictedCount() {
184       return lruCacheStats.getEvictedCount()
185           + bucketCacheStats.getEvictedCount();
186     }
187 
188     @Override
189     public void rollMetricsPeriod() {
190       lruCacheStats.rollMetricsPeriod();
191       bucketCacheStats.rollMetricsPeriod();
192     }
193     
194     @Override
195     public long getSumHitCountsPastNPeriods() {
196       return lruCacheStats.getSumHitCountsPastNPeriods()
197           + bucketCacheStats.getSumHitCountsPastNPeriods();
198     }
199     
200     @Override
201     public long getSumRequestCountsPastNPeriods() {
202       return lruCacheStats.getSumRequestCountsPastNPeriods()
203           + bucketCacheStats.getSumRequestCountsPastNPeriods();
204     }
205     
206     @Override
207     public long getSumHitCachingCountsPastNPeriods() {
208       return lruCacheStats.getSumHitCachingCountsPastNPeriods()
209           + bucketCacheStats.getSumHitCachingCountsPastNPeriods();
210     }
211 
212     @Override
213     public long getSumRequestCachingCountsPastNPeriods() {
214       return lruCacheStats.getSumRequestCachingCountsPastNPeriods()
215           + bucketCacheStats.getSumRequestCachingCountsPastNPeriods();
216     }
217   }
218 
219   @Override
220   public Iterator<CachedBlock> iterator() {
221     return new BlockCachesIterator(getBlockCaches());
222   }
223 
224   @Override
225   public BlockCache[] getBlockCaches() {
226     return new BlockCache [] {this.lruCache, this.l2Cache};
227   }
228 
229   @Override
230   public void setMaxSize(long size) {
231     this.lruCache.setMaxSize(size);
232   }
233 }