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