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