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 */ 018 019package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; 020 021import com.google.common.base.Preconditions; 022import com.google.common.util.concurrent.ThreadFactoryBuilder; 023 024import java.io.FileInputStream; 025import java.io.FileNotFoundException; 026import java.io.IOException; 027import java.util.ArrayList; 028import java.util.HashMap; 029import java.util.Iterator; 030import java.util.List; 031import java.util.Map.Entry; 032import java.util.concurrent.Executor; 033import java.util.concurrent.LinkedBlockingQueue; 034import java.util.concurrent.ThreadFactory; 035import java.util.concurrent.ThreadPoolExecutor; 036import java.util.concurrent.TimeUnit; 037import java.util.concurrent.atomic.AtomicLong; 038 039import org.apache.commons.io.IOUtils; 040import org.apache.commons.lang.builder.HashCodeBuilder; 041import org.apache.commons.logging.Log; 042import org.apache.commons.logging.LogFactory; 043import org.apache.hadoop.classification.InterfaceAudience; 044import org.apache.hadoop.classification.InterfaceStability; 045import org.apache.hadoop.fs.ChecksumException; 046import org.apache.hadoop.hdfs.DFSConfigKeys; 047import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; 048import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 049import org.apache.hadoop.io.nativeio.NativeIO; 050 051/** 052 * Manages caching for an FsDatasetImpl by using the mmap(2) and mlock(2) 053 * system calls to lock blocks into memory. Block checksums are verified upon 054 * entry into the cache. 055 */ 056@InterfaceAudience.Private 057@InterfaceStability.Unstable 058public class FsDatasetCache { 059 /** 060 * Keys which identify MappableBlocks. 061 */ 062 private static final class Key { 063 /** 064 * Block id. 065 */ 066 final long id; 067 068 /** 069 * Block pool id. 070 */ 071 final String bpid; 072 073 Key(long id, String bpid) { 074 this.id = id; 075 this.bpid = bpid; 076 } 077 078 @Override 079 public boolean equals(Object o) { 080 if (o == null) { 081 return false; 082 } 083 if (!(o.getClass() == getClass())) { 084 return false; 085 } 086 Key other = (Key)o; 087 return ((other.id == this.id) && (other.bpid.equals(this.bpid))); 088 } 089 090 @Override 091 public int hashCode() { 092 return new HashCodeBuilder().append(id).append(bpid).hashCode(); 093 } 094 }; 095 096 /** 097 * MappableBlocks that we know about. 098 */ 099 private static final class Value { 100 final State state; 101 final MappableBlock mappableBlock; 102 103 Value(MappableBlock mappableBlock, State state) { 104 this.mappableBlock = mappableBlock; 105 this.state = state; 106 } 107 } 108 109 private enum State { 110 /** 111 * The MappableBlock is in the process of being cached. 112 */ 113 CACHING, 114 115 /** 116 * The MappableBlock was in the process of being cached, but it was 117 * cancelled. Only the FsDatasetCache#WorkerTask can remove cancelled 118 * MappableBlock objects. 119 */ 120 CACHING_CANCELLED, 121 122 /** 123 * The MappableBlock is in the cache. 124 */ 125 CACHED, 126 127 /** 128 * The MappableBlock is in the process of uncaching. 129 */ 130 UNCACHING; 131 132 /** 133 * Whether we should advertise this block as cached to the NameNode and 134 * clients. 135 */ 136 public boolean shouldAdvertise() { 137 return (this == CACHED); 138 } 139 } 140 141 private static final Log LOG = LogFactory.getLog(FsDatasetCache.class); 142 143 /** 144 * Stores MappableBlock objects and the states they're in. 145 */ 146 private final HashMap<Key, Value> mappableBlockMap = new HashMap<Key, Value>(); 147 148 private final AtomicLong numBlocksCached = new AtomicLong(0); 149 150 private final FsDatasetImpl dataset; 151 152 private final ThreadPoolExecutor uncachingExecutor; 153 154 /** 155 * The approximate amount of cache space in use. 156 * 157 * This number is an overestimate, counting bytes that will be used only 158 * if pending caching operations succeed. It does not take into account 159 * pending uncaching operations. 160 * 161 * This overestimate is more useful to the NameNode than an underestimate, 162 * since we don't want the NameNode to assign us more replicas than 163 * we can cache, because of the current batch of operations. 164 */ 165 private final UsedBytesCount usedBytesCount; 166 167 public static class PageRounder { 168 private final long osPageSize = 169 NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize(); 170 171 /** 172 * Round up a number to the operating system page size. 173 */ 174 public long round(long count) { 175 long newCount = 176 (count + (osPageSize - 1)) / osPageSize; 177 return newCount * osPageSize; 178 } 179 } 180 181 private class UsedBytesCount { 182 private final AtomicLong usedBytes = new AtomicLong(0); 183 184 private PageRounder rounder = new PageRounder(); 185 186 /** 187 * Try to reserve more bytes. 188 * 189 * @param count The number of bytes to add. We will round this 190 * up to the page size. 191 * 192 * @return The new number of usedBytes if we succeeded; 193 * -1 if we failed. 194 */ 195 long reserve(long count) { 196 count = rounder.round(count); 197 while (true) { 198 long cur = usedBytes.get(); 199 long next = cur + count; 200 if (next > maxBytes) { 201 return -1; 202 } 203 if (usedBytes.compareAndSet(cur, next)) { 204 return next; 205 } 206 } 207 } 208 209 /** 210 * Release some bytes that we're using. 211 * 212 * @param count The number of bytes to release. We will round this 213 * up to the page size. 214 * 215 * @return The new number of usedBytes. 216 */ 217 long release(long count) { 218 count = rounder.round(count); 219 return usedBytes.addAndGet(-count); 220 } 221 222 long get() { 223 return usedBytes.get(); 224 } 225 } 226 227 /** 228 * The total cache capacity in bytes. 229 */ 230 private final long maxBytes; 231 232 /** 233 * Number of cache commands that could not be completed successfully 234 */ 235 AtomicLong numBlocksFailedToCache = new AtomicLong(0); 236 /** 237 * Number of uncache commands that could not be completed successfully 238 */ 239 AtomicLong numBlocksFailedToUncache = new AtomicLong(0); 240 241 public FsDatasetCache(FsDatasetImpl dataset) { 242 this.dataset = dataset; 243 this.maxBytes = dataset.datanode.getDnConf().getMaxLockedMemory(); 244 ThreadFactory workerFactory = new ThreadFactoryBuilder() 245 .setDaemon(true) 246 .setNameFormat("FsDatasetCache-%d-" + dataset.toString()) 247 .build(); 248 this.usedBytesCount = new UsedBytesCount(); 249 this.uncachingExecutor = new ThreadPoolExecutor( 250 0, 1, 251 60, TimeUnit.SECONDS, 252 new LinkedBlockingQueue<Runnable>(), 253 workerFactory); 254 this.uncachingExecutor.allowCoreThreadTimeOut(true); 255 } 256 257 /** 258 * @return List of cached blocks suitable for translation into a 259 * {@link BlockListAsLongs} for a cache report. 260 */ 261 synchronized List<Long> getCachedBlocks(String bpid) { 262 List<Long> blocks = new ArrayList<Long>(); 263 for (Iterator<Entry<Key, Value>> iter = 264 mappableBlockMap.entrySet().iterator(); iter.hasNext(); ) { 265 Entry<Key, Value> entry = iter.next(); 266 if (entry.getKey().bpid.equals(bpid)) { 267 if (entry.getValue().state.shouldAdvertise()) { 268 blocks.add(entry.getKey().id); 269 } 270 } 271 } 272 return blocks; 273 } 274 275 /** 276 * Attempt to begin caching a block. 277 */ 278 synchronized void cacheBlock(long blockId, String bpid, 279 String blockFileName, long length, long genstamp, 280 Executor volumeExecutor) { 281 Key key = new Key(blockId, bpid); 282 Value prevValue = mappableBlockMap.get(key); 283 if (prevValue != null) { 284 if (LOG.isDebugEnabled()) { 285 LOG.debug("Block with id " + blockId + ", pool " + bpid + 286 " already exists in the FsDatasetCache with state " + 287 prevValue.state); 288 } 289 numBlocksFailedToCache.incrementAndGet(); 290 return; 291 } 292 mappableBlockMap.put(key, new Value(null, State.CACHING)); 293 volumeExecutor.execute( 294 new CachingTask(key, blockFileName, length, genstamp)); 295 if (LOG.isDebugEnabled()) { 296 LOG.debug("Initiating caching for Block with id " + blockId + 297 ", pool " + bpid); 298 } 299 } 300 301 synchronized void uncacheBlock(String bpid, long blockId) { 302 Key key = new Key(blockId, bpid); 303 Value prevValue = mappableBlockMap.get(key); 304 305 if (prevValue == null) { 306 if (LOG.isDebugEnabled()) { 307 LOG.debug("Block with id " + blockId + ", pool " + bpid + " " + 308 "does not need to be uncached, because it is not currently " + 309 "in the mappableBlockMap."); 310 } 311 numBlocksFailedToUncache.incrementAndGet(); 312 return; 313 } 314 switch (prevValue.state) { 315 case CACHING: 316 if (LOG.isDebugEnabled()) { 317 LOG.debug("Cancelling caching for block with id " + blockId + 318 ", pool " + bpid + "."); 319 } 320 mappableBlockMap.put(key, 321 new Value(prevValue.mappableBlock, State.CACHING_CANCELLED)); 322 break; 323 case CACHED: 324 if (LOG.isDebugEnabled()) { 325 LOG.debug("Block with id " + blockId + ", pool " + bpid + " " + 326 "has been scheduled for uncaching."); 327 } 328 mappableBlockMap.put(key, 329 new Value(prevValue.mappableBlock, State.UNCACHING)); 330 uncachingExecutor.execute(new UncachingTask(key)); 331 break; 332 default: 333 if (LOG.isDebugEnabled()) { 334 LOG.debug("Block with id " + blockId + ", pool " + bpid + " " + 335 "does not need to be uncached, because it is " + 336 "in state " + prevValue.state + "."); 337 } 338 numBlocksFailedToUncache.incrementAndGet(); 339 break; 340 } 341 } 342 343 /** 344 * Background worker that mmaps, mlocks, and checksums a block 345 */ 346 private class CachingTask implements Runnable { 347 private final Key key; 348 private final String blockFileName; 349 private final long length; 350 private final long genstamp; 351 352 CachingTask(Key key, String blockFileName, long length, long genstamp) { 353 this.key = key; 354 this.blockFileName = blockFileName; 355 this.length = length; 356 this.genstamp = genstamp; 357 } 358 359 @Override 360 public void run() { 361 boolean success = false; 362 FileInputStream blockIn = null, metaIn = null; 363 MappableBlock mappableBlock = null; 364 ExtendedBlock extBlk = 365 new ExtendedBlock(key.bpid, key.id, length, genstamp); 366 long newUsedBytes = usedBytesCount.reserve(length); 367 if (newUsedBytes < 0) { 368 LOG.warn("Failed to cache block id " + key.id + ", pool " + key.bpid + 369 ": could not reserve " + length + " more bytes in the " + 370 "cache: " + DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY + 371 " of " + maxBytes + " exceeded."); 372 numBlocksFailedToCache.incrementAndGet(); 373 return; 374 } 375 try { 376 try { 377 blockIn = (FileInputStream)dataset.getBlockInputStream(extBlk, 0); 378 metaIn = (FileInputStream)dataset.getMetaDataInputStream(extBlk) 379 .getWrappedStream(); 380 } catch (ClassCastException e) { 381 LOG.warn("Failed to cache block with id " + key.id + ", pool " + 382 key.bpid + ": Underlying blocks are not backed by files.", e); 383 return; 384 } catch (FileNotFoundException e) { 385 LOG.info("Failed to cache block with id " + key.id + ", pool " + 386 key.bpid + ": failed to find backing files."); 387 return; 388 } catch (IOException e) { 389 LOG.warn("Failed to cache block with id " + key.id + ", pool " + 390 key.bpid + ": failed to open file", e); 391 return; 392 } 393 try { 394 mappableBlock = MappableBlock. 395 load(length, blockIn, metaIn, blockFileName); 396 } catch (ChecksumException e) { 397 // Exception message is bogus since this wasn't caused by a file read 398 LOG.warn("Failed to cache block " + key.id + " in " + key.bpid + ": " + 399 "checksum verification failed."); 400 return; 401 } catch (IOException e) { 402 LOG.warn("Failed to cache block " + key.id + " in " + key.bpid, e); 403 return; 404 } 405 synchronized (FsDatasetCache.this) { 406 Value value = mappableBlockMap.get(key); 407 Preconditions.checkNotNull(value); 408 Preconditions.checkState(value.state == State.CACHING || 409 value.state == State.CACHING_CANCELLED); 410 if (value.state == State.CACHING_CANCELLED) { 411 mappableBlockMap.remove(key); 412 LOG.warn("Caching of block " + key.id + " in " + key.bpid + 413 " was cancelled."); 414 return; 415 } 416 mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED)); 417 } 418 if (LOG.isDebugEnabled()) { 419 LOG.debug("Successfully cached block " + key.id + " in " + key.bpid + 420 ". We are now caching " + newUsedBytes + " bytes in total."); 421 } 422 numBlocksCached.addAndGet(1); 423 success = true; 424 } finally { 425 if (!success) { 426 newUsedBytes = usedBytesCount.release(length); 427 if (LOG.isDebugEnabled()) { 428 LOG.debug("Caching of block " + key.id + " in " + 429 key.bpid + " was aborted. We are now caching only " + 430 newUsedBytes + " + bytes in total."); 431 } 432 IOUtils.closeQuietly(blockIn); 433 IOUtils.closeQuietly(metaIn); 434 if (mappableBlock != null) { 435 mappableBlock.close(); 436 } 437 numBlocksFailedToCache.incrementAndGet(); 438 439 synchronized (FsDatasetCache.this) { 440 mappableBlockMap.remove(key); 441 } 442 } 443 } 444 } 445 } 446 447 private class UncachingTask implements Runnable { 448 private final Key key; 449 450 UncachingTask(Key key) { 451 this.key = key; 452 } 453 454 @Override 455 public void run() { 456 Value value; 457 458 synchronized (FsDatasetCache.this) { 459 value = mappableBlockMap.get(key); 460 } 461 Preconditions.checkNotNull(value); 462 Preconditions.checkArgument(value.state == State.UNCACHING); 463 // TODO: we will eventually need to do revocation here if any clients 464 // are reading via mmap with checksums enabled. See HDFS-5182. 465 IOUtils.closeQuietly(value.mappableBlock); 466 synchronized (FsDatasetCache.this) { 467 mappableBlockMap.remove(key); 468 } 469 long newUsedBytes = 470 usedBytesCount.release(value.mappableBlock.getLength()); 471 numBlocksCached.addAndGet(-1); 472 if (LOG.isDebugEnabled()) { 473 LOG.debug("Uncaching of block " + key.id + " in " + key.bpid + 474 " completed. usedBytes = " + newUsedBytes); 475 } 476 } 477 } 478 479 // Stats related methods for FSDatasetMBean 480 481 /** 482 * Get the approximate amount of cache space used. 483 */ 484 public long getCacheUsed() { 485 return usedBytesCount.get(); 486 } 487 488 /** 489 * Get the maximum amount of bytes we can cache. This is a constant. 490 */ 491 public long getCacheCapacity() { 492 return maxBytes; 493 } 494 495 public long getNumBlocksFailedToCache() { 496 return numBlocksFailedToCache.get(); 497 } 498 499 public long getNumBlocksFailedToUncache() { 500 return numBlocksFailedToUncache.get(); 501 } 502 503 public long getNumBlocksCached() { 504 return numBlocksCached.get(); 505 } 506}