001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.hfile;
019
020import java.io.IOException;
021import java.nio.ByteBuffer;
022import java.util.NavigableMap;
023import java.util.NavigableSet;
024import java.util.concurrent.ConcurrentSkipListMap;
025import java.util.concurrent.ConcurrentSkipListSet;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
028import org.apache.hadoop.hbase.util.Bytes;
029import org.apache.hadoop.hbase.util.GsonUtil;
030import org.apache.yetus.audience.InterfaceAudience;
031import org.slf4j.Logger;
032import org.slf4j.LoggerFactory;
033
034import org.apache.hbase.thirdparty.com.google.gson.Gson;
035import org.apache.hbase.thirdparty.com.google.gson.TypeAdapter;
036import org.apache.hbase.thirdparty.com.google.gson.stream.JsonReader;
037import org.apache.hbase.thirdparty.com.google.gson.stream.JsonWriter;
038
039/**
040 * Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
041 * No attempt has been made at making this thread safe.
042 */
043@InterfaceAudience.Private
044public class BlockCacheUtil {
045
046  private static final Logger LOG = LoggerFactory.getLogger(BlockCacheUtil.class);
047
048  public static final long NANOS_PER_SECOND = 1000000000;
049
050  /**
051   * Needed generating JSON.
052   */
053  private static final Gson GSON = GsonUtil.createGson()
054    .registerTypeAdapter(FastLongHistogram.class, new TypeAdapter<FastLongHistogram>() {
055
056      @Override
057      public void write(JsonWriter out, FastLongHistogram value) throws IOException {
058        AgeSnapshot snapshot = new AgeSnapshot(value);
059        out.beginObject();
060        out.name("mean").value(snapshot.getMean());
061        out.name("min").value(snapshot.getMin());
062        out.name("max").value(snapshot.getMax());
063        out.name("75thPercentile").value(snapshot.get75thPercentile());
064        out.name("95thPercentile").value(snapshot.get95thPercentile());
065        out.name("98thPercentile").value(snapshot.get98thPercentile());
066        out.name("99thPercentile").value(snapshot.get99thPercentile());
067        out.name("999thPercentile").value(snapshot.get999thPercentile());
068        out.endObject();
069      }
070
071      @Override
072      public FastLongHistogram read(JsonReader in) throws IOException {
073        throw new UnsupportedOperationException();
074      }
075    }).setPrettyPrinting().create();
076
077  /**
078   * @param cb
079   * @return The block content as String.
080   */
081  public static String toString(final CachedBlock cb, final long now) {
082    return "filename=" + cb.getFilename() + ", " + toStringMinusFileName(cb, now);
083  }
084
085  /**
086   * Little data structure to hold counts for a file.
087   * Used doing a toJSON.
088   */
089  static class CachedBlockCountsPerFile {
090    private int count = 0;
091    private long size = 0;
092    private int countData = 0;
093    private long sizeData = 0;
094    private final String filename;
095
096    CachedBlockCountsPerFile(final String filename) {
097      this.filename = filename;
098    }
099
100    public int getCount() {
101      return count;
102    }
103
104    public long getSize() {
105      return size;
106    }
107
108    public int getCountData() {
109      return countData;
110    }
111
112    public long getSizeData() {
113      return sizeData;
114    }
115
116    public String getFilename() {
117      return filename;
118    }
119  }
120
121  /**
122   * @return A JSON String of <code>filename</code> and counts of <code>blocks</code>
123   */
124  public static String toJSON(String filename, NavigableSet<CachedBlock> blocks)
125      throws IOException {
126    CachedBlockCountsPerFile counts = new CachedBlockCountsPerFile(filename);
127    for (CachedBlock cb : blocks) {
128      counts.count++;
129      counts.size += cb.getSize();
130      BlockType bt = cb.getBlockType();
131      if (bt != null && bt.isData()) {
132        counts.countData++;
133        counts.sizeData += cb.getSize();
134      }
135    }
136    return GSON.toJson(counts);
137  }
138
139  /**
140   * @return JSON string of <code>cbsf</code> aggregated
141   */
142  public static String toJSON(CachedBlocksByFile cbsbf) throws IOException {
143    return GSON.toJson(cbsbf);
144  }
145
146  /**
147   * @return JSON string of <code>bc</code> content.
148   */
149  public static String toJSON(BlockCache bc) throws IOException {
150    return GSON.toJson(bc);
151  }
152
153  /**
154   * @param cb
155   * @return The block content of <code>bc</code> as a String minus the filename.
156   */
157  public static String toStringMinusFileName(final CachedBlock cb, final long now) {
158    return "offset=" + cb.getOffset() +
159      ", size=" + cb.getSize() +
160      ", age=" + (now - cb.getCachedTime()) +
161      ", type=" + cb.getBlockType() +
162      ", priority=" + cb.getBlockPriority();
163  }
164
165  /**
166   * Get a {@link CachedBlocksByFile} instance and load it up by iterating content in
167   * {@link BlockCache}.
168   * @param conf Used to read configurations
169   * @param bc Block Cache to iterate.
170   * @return Laoded up instance of CachedBlocksByFile
171   */
172  public static CachedBlocksByFile getLoadedCachedBlocksByFile(final Configuration conf,
173      final BlockCache bc) {
174    CachedBlocksByFile cbsbf = new CachedBlocksByFile(conf);
175    for (CachedBlock cb: bc) {
176      if (cbsbf.update(cb)) break;
177    }
178    return cbsbf;
179  }
180
181  private static int compareCacheBlock(Cacheable left, Cacheable right,
182                                       boolean includeNextBlockMetadata) {
183    ByteBuffer l = ByteBuffer.allocate(left.getSerializedLength());
184    left.serialize(l, includeNextBlockMetadata);
185    ByteBuffer r = ByteBuffer.allocate(right.getSerializedLength());
186    right.serialize(r, includeNextBlockMetadata);
187    return Bytes.compareTo(l.array(), l.arrayOffset(), l.limit(),
188        r.array(), r.arrayOffset(), r.limit());
189  }
190
191  /**
192   * Validate that the existing and newBlock are the same without including the nextBlockMetadata,
193   * if not, throw an exception. If they are the same without the nextBlockMetadata,
194   * return the comparison.
195   *
196   * @param existing block that is existing in the cache.
197   * @param newBlock block that is trying to be cached.
198   * @param cacheKey the cache key of the blocks.
199   * @return comparison of the existing block to the newBlock.
200   */
201  public static int validateBlockAddition(Cacheable existing, Cacheable newBlock,
202                                          BlockCacheKey cacheKey) {
203    int comparison = compareCacheBlock(existing, newBlock, false);
204    if (comparison != 0) {
205      throw new RuntimeException("Cached block contents differ, which should not have happened."
206                                 + "cacheKey:" + cacheKey);
207    }
208    if ((existing instanceof HFileBlock) && (newBlock instanceof HFileBlock)) {
209      comparison = ((HFileBlock) existing).getNextBlockOnDiskSize()
210          - ((HFileBlock) newBlock).getNextBlockOnDiskSize();
211    }
212    return comparison;
213  }
214
215  /**
216   * Because of the region splitting, it's possible that the split key locate in the middle of a
217   * block. So it's possible that both the daughter regions load the same block from their parent
218   * HFile. When pread, we don't force the read to read all of the next block header. So when two
219   * threads try to cache the same block, it's possible that one thread read all of the next block
220   * header but the other one didn't. if the already cached block hasn't next block header but the
221   * new block to cache has, then we can replace the existing block with the new block for better
222   * performance.(HBASE-20447)
223   * @param blockCache BlockCache to check
224   * @param cacheKey the block cache key
225   * @param newBlock the new block which try to put into the block cache.
226   * @return true means need to replace existing block with new block for the same block cache key.
227   *         false means just keep the existing block.
228   */
229  public static boolean shouldReplaceExistingCacheBlock(BlockCache blockCache,
230      BlockCacheKey cacheKey, Cacheable newBlock) {
231    Cacheable existingBlock = blockCache.getBlock(cacheKey, false, false, false);
232    if (null == existingBlock) {
233      // Not exist now.
234      return true;
235    }
236    try {
237      int comparison = BlockCacheUtil.validateBlockAddition(existingBlock, newBlock, cacheKey);
238      if (comparison < 0) {
239        LOG.warn("Cached block contents differ by nextBlockOnDiskSize, the new block has "
240            + "nextBlockOnDiskSize set. Caching new block.");
241        return true;
242      } else if (comparison > 0) {
243        LOG.warn("Cached block contents differ by nextBlockOnDiskSize, the existing block has "
244            + "nextBlockOnDiskSize set, Keeping cached block.");
245        return false;
246      } else {
247        LOG.warn("Caching an already cached block: {}. This is harmless and can happen in rare "
248            + "cases (see HBASE-8547)",
249          cacheKey);
250        return false;
251      }
252    } finally {
253      // return the block since we need to decrement the count
254      blockCache.returnBlock(cacheKey, existingBlock);
255    }
256  }
257
258  /**
259   * Use one of these to keep a running account of cached blocks by file.  Throw it away when done.
260   * This is different than metrics in that it is stats on current state of a cache.
261   * See getLoadedCachedBlocksByFile
262   */
263  public static class CachedBlocksByFile {
264    private int count;
265    private int dataBlockCount;
266    private long size;
267    private long dataSize;
268    private final long now = System.nanoTime();
269    /**
270     * How many blocks to look at before we give up.
271     * There could be many millions of blocks. We don't want the
272     * ui to freeze while we run through 1B blocks... users will
273     * think hbase dead. UI displays warning in red when stats
274     * are incomplete.
275     */
276    private final int max;
277    public static final int DEFAULT_MAX = 1000000;
278
279    CachedBlocksByFile() {
280      this(null);
281    }
282
283    CachedBlocksByFile(final Configuration c) {
284      this.max = c == null? DEFAULT_MAX: c.getInt("hbase.ui.blockcache.by.file.max", DEFAULT_MAX);
285    }
286
287    /**
288     * Map by filename. use concurent utils because we want our Map and contained blocks sorted.
289     */
290    private transient NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
291      new ConcurrentSkipListMap<>();
292    FastLongHistogram hist = new FastLongHistogram();
293
294    /**
295     * @param cb
296     * @return True if full.... if we won't be adding any more.
297     */
298    public boolean update(final CachedBlock cb) {
299      if (isFull()) return true;
300      NavigableSet<CachedBlock> set = this.cachedBlockByFile.get(cb.getFilename());
301      if (set == null) {
302        set = new ConcurrentSkipListSet<>();
303        this.cachedBlockByFile.put(cb.getFilename(), set);
304      }
305      set.add(cb);
306      this.size += cb.getSize();
307      this.count++;
308      BlockType bt = cb.getBlockType();
309      if (bt != null && bt.isData()) {
310        this.dataBlockCount++;
311        this.dataSize += cb.getSize();
312      }
313      long age = (this.now - cb.getCachedTime())/NANOS_PER_SECOND;
314      this.hist.add(age, 1);
315      return false;
316    }
317
318    /**
319     * @return True if full; i.e. there are more items in the cache but we only loaded up
320     * the maximum set in configuration <code>hbase.ui.blockcache.by.file.max</code>
321     * (Default: DEFAULT_MAX).
322     */
323    public boolean isFull() {
324      return this.count >= this.max;
325    }
326
327    public NavigableMap<String, NavigableSet<CachedBlock>> getCachedBlockStatsByFile() {
328      return this.cachedBlockByFile;
329    }
330
331    /**
332     * @return count of blocks in the cache
333     */
334    public int getCount() {
335      return count;
336    }
337
338    public int getDataCount() {
339      return dataBlockCount;
340    }
341
342    /**
343     * @return size of blocks in the cache
344     */
345    public long getSize() {
346      return size;
347    }
348
349    /**
350     * @return Size of data.
351     */
352    public long getDataSize() {
353      return dataSize;
354    }
355
356    public AgeSnapshot getAgeInCacheSnapshot() {
357      return new AgeSnapshot(this.hist);
358    }
359
360    @Override
361    public String toString() {
362      AgeSnapshot snapshot = getAgeInCacheSnapshot();
363      return "count=" + count + ", dataBlockCount=" + dataBlockCount + ", size=" + size +
364          ", dataSize=" + getDataSize() +
365          ", mean age=" + snapshot.getMean() +
366          ", min age=" + snapshot.getMin() +
367          ", max age=" + snapshot.getMax() +
368          ", 75th percentile age="   + snapshot.get75thPercentile() +
369          ", 95th percentile age="   + snapshot.get95thPercentile() +
370          ", 98th percentile age="   + snapshot.get98thPercentile() +
371          ", 99th percentile age="   + snapshot.get99thPercentile() +
372          ", 99.9th percentile age=" + snapshot.get99thPercentile();
373    }
374  }
375}