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 org.apache.hadoop.conf.Configuration;
022import org.apache.hadoop.fs.Path;
023import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
024import org.apache.yetus.audience.InterfaceAudience;
025import org.slf4j.Logger;
026import org.slf4j.LoggerFactory;
027
028/**
029 * Implementation of {@link HFile.Reader} to deal with pread.
030 */
031@InterfaceAudience.Private
032public class HFilePreadReader extends HFileReaderImpl {
033  private static final Logger LOG = LoggerFactory.getLogger(HFileReaderImpl.class);
034
035  private static final int WAIT_TIME_FOR_CACHE_INITIALIZATION = 10 * 60 * 1000;
036
037  public HFilePreadReader(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf,
038    Configuration conf) throws IOException {
039    super(context, fileInfo, cacheConf, conf);
040    // Initialize HFileInfo object with metadata for caching decisions
041    fileInfo.initMetaAndIndex(this);
042    // master hosted regions, like the master procedures store wouldn't have a block cache
043    // Prefetch file blocks upon open if requested
044    if (cacheConf.getBlockCache().isPresent() && cacheConf.shouldPrefetchOnOpen()) {
045      PrefetchExecutor.request(path, new Runnable() {
046        @Override
047        public void run() {
048          long offset = 0;
049          long end = 0;
050          HFile.Reader prefetchStreamReader = null;
051          try {
052            cacheConf.getBlockCache().ifPresent(
053              cache -> cache.waitForCacheInitialization(WAIT_TIME_FOR_CACHE_INITIALIZATION));
054            ReaderContext streamReaderContext = ReaderContextBuilder.newBuilder(context)
055              .withReaderType(ReaderContext.ReaderType.STREAM)
056              .withInputStreamWrapper(new FSDataInputStreamWrapper(context.getFileSystem(),
057                context.getInputStreamWrapper().getReaderPath()))
058              .build();
059            prefetchStreamReader =
060              new HFileStreamReader(streamReaderContext, fileInfo, cacheConf, conf);
061            end = getTrailer().getLoadOnOpenDataOffset();
062            if (LOG.isTraceEnabled()) {
063              LOG.trace("Prefetch start " + getPathOffsetEndStr(path, offset, end));
064            }
065            // Don't use BlockIterator here, because it's designed to read load-on-open section.
066            long onDiskSizeOfNextBlock = -1;
067            // if we are here, block cache is present anyways
068            BlockCache cache = cacheConf.getBlockCache().get();
069            boolean interrupted = false;
070            int blockCount = 0;
071            int dataBlockCount = 0;
072            while (offset < end) {
073              if (Thread.interrupted()) {
074                break;
075              }
076              // Some cache implementations can be persistent and resilient to restarts,
077              // so we check first if the block exists on its in-memory index, if so, we just
078              // update the offset and move on to the next block without actually going read all
079              // the way to the cache.
080              BlockCacheKey cacheKey = new BlockCacheKey(name, offset);
081              if (cache.isAlreadyCached(cacheKey).orElse(false)) {
082                // Right now, isAlreadyCached is only supported by BucketCache, which should
083                // always cache data blocks.
084                int size = cache.getBlockSize(cacheKey).orElse(0);
085                if (size > 0) {
086                  offset += size;
087                  LOG.debug("Found block of size {} for cache key {}. "
088                    + "Skipping prefetch, the block is already cached.", size, cacheKey);
089                  blockCount++;
090                  dataBlockCount++;
091                  // We need to reset this here, because we don't know the size of next block, since
092                  // we never recovered the current block.
093                  onDiskSizeOfNextBlock = -1;
094                  continue;
095                } else {
096                  LOG.debug("Found block for cache key {}, but couldn't get its size. "
097                    + "Maybe the cache implementation doesn't support it? "
098                    + "We'll need to read the block from cache or file system. ", cacheKey);
099                }
100              } else {
101                LOG.debug("No entry in the backing map for cache key {}. ", cacheKey);
102              }
103              // Perhaps we got our block from cache? Unlikely as this may be, if it happens, then
104              // the internal-to-hfileblock thread local which holds the overread that gets the
105              // next header, will not have happened...so, pass in the onDiskSize gotten from the
106              // cached block. This 'optimization' triggers extremely rarely I'd say.
107              HFileBlock block = prefetchStreamReader.readBlock(offset, onDiskSizeOfNextBlock,
108                /* cacheBlock= */true, /* pread= */false, false, false, null, null, true);
109              try {
110                if (!cacheConf.isInMemory()) {
111                  if (!cache.blockFitsIntoTheCache(block).orElse(true)) {
112                    LOG.warn(
113                      "Interrupting prefetch for file {} because block {} of size {} "
114                        + "doesn't fit in the available cache space. isCacheEnabled: {}",
115                      path, cacheKey, block.getOnDiskSizeWithHeader(), cache.isCacheEnabled());
116                    interrupted = true;
117                    break;
118                  }
119                  if (!cacheConf.isHeapUsageBelowThreshold()) {
120                    LOG.warn(
121                      "Interrupting prefetch because heap usage is above the threshold: {} "
122                        + "configured via {}",
123                      cacheConf.getHeapUsageThreshold(), CacheConfig.PREFETCH_HEAP_USAGE_THRESHOLD);
124                    interrupted = true;
125                    break;
126                  }
127                }
128                onDiskSizeOfNextBlock = block.getNextBlockOnDiskSize();
129                offset += block.getOnDiskSizeWithHeader();
130                blockCount++;
131                if (block.getBlockType().isData()) {
132                  dataBlockCount++;
133                }
134              } finally {
135                // Ideally here the readBlock won't find the block in cache. We call this
136                // readBlock so that block data is read from FS and cached in BC. we must call
137                // returnBlock here to decrease the reference count of block.
138                block.release();
139              }
140            }
141            if (!interrupted) {
142              cacheConf.getBlockCache().get().notifyFileCachingCompleted(path, blockCount,
143                dataBlockCount, offset);
144            }
145          } catch (IOException e) {
146            // IOExceptions are probably due to region closes (relocation, etc.)
147            if (LOG.isDebugEnabled()) {
148              LOG.debug("Prefetch " + getPathOffsetEndStr(path, offset, end), e);
149            }
150          } catch (Throwable e) {
151            // Other exceptions are interesting
152            LOG.warn("Prefetch " + getPathOffsetEndStr(path, offset, end), e);
153          } finally {
154            if (prefetchStreamReader != null) {
155              try {
156                prefetchStreamReader.close(false);
157              } catch (IOException e) {
158                LOG.warn("Close prefetch stream reader failed, path: " + path, e);
159              }
160            }
161            PrefetchExecutor.complete(path);
162          }
163        }
164      });
165    }
166  }
167
168  /*
169   * Get the region name for the given file path. A HFile is always kept under the <region>/<column
170   * family>/<hfile>. To find the region for a given hFile, just find the name of the grandparent
171   * directory.
172   */
173  private static String getRegionName(Path path) {
174    return path.getParent().getParent().getName();
175  }
176
177  private static String getPathOffsetEndStr(final Path path, final long offset, final long end) {
178    return "path=" + path.toString() + ", offset=" + offset + ", end=" + end;
179  }
180
181  public void close(boolean evictOnClose) throws IOException {
182    PrefetchExecutor.cancel(path);
183    // Deallocate blocks in load-on-open section
184    this.fileInfo.close();
185    // Deallocate data blocks
186    cacheConf.getBlockCache().ifPresent(cache -> {
187      if (evictOnClose) {
188        int numEvicted = cache.evictBlocksByHfileName(name);
189        if (LOG.isTraceEnabled()) {
190          LOG.trace("On close, file= {} evicted= {} block(s)", name, numEvicted);
191        }
192      }
193    });
194    fsBlockReader.closeStreams();
195  }
196}