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}