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}