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}