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 static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS; 022import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS_DEFAULT; 023import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_CACHE_REVOCATION_POLLING_MS; 024import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_CACHE_REVOCATION_POLLING_MS_DEFAULT; 025 026import com.google.common.base.Preconditions; 027import com.google.common.util.concurrent.ThreadFactoryBuilder; 028 029import java.io.FileInputStream; 030import java.io.FileNotFoundException; 031import java.io.IOException; 032import java.util.ArrayList; 033import java.util.HashMap; 034import java.util.Iterator; 035import java.util.List; 036import java.util.Map.Entry; 037import java.util.concurrent.Executor; 038import java.util.concurrent.LinkedBlockingQueue; 039import java.util.concurrent.ThreadFactory; 040import java.util.concurrent.ThreadPoolExecutor; 041import java.util.concurrent.ScheduledThreadPoolExecutor; 042import java.util.concurrent.TimeUnit; 043import java.util.concurrent.atomic.AtomicLong; 044 045import org.apache.commons.io.IOUtils; 046import org.apache.commons.lang.time.DurationFormatUtils; 047import org.apache.hadoop.classification.InterfaceAudience; 048import org.apache.hadoop.classification.InterfaceStability; 049import org.apache.hadoop.fs.ChecksumException; 050import org.apache.hadoop.hdfs.ExtendedBlockId; 051import org.apache.hadoop.hdfs.DFSConfigKeys; 052import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; 053import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 054import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil; 055import org.apache.hadoop.io.nativeio.NativeIO; 056import org.apache.hadoop.util.Time; 057import org.slf4j.Logger; 058import org.slf4j.LoggerFactory; 059 060/** 061 * Manages caching for an FsDatasetImpl by using the mmap(2) and mlock(2) 062 * system calls to lock blocks into memory. Block checksums are verified upon 063 * entry into the cache. 064 */ 065@InterfaceAudience.Private 066@InterfaceStability.Unstable 067public class FsDatasetCache { 068 /** 069 * MappableBlocks that we know about. 070 */ 071 private static final class Value { 072 final State state; 073 final MappableBlock mappableBlock; 074 075 Value(MappableBlock mappableBlock, State state) { 076 this.mappableBlock = mappableBlock; 077 this.state = state; 078 } 079 } 080 081 private enum State { 082 /** 083 * The MappableBlock is in the process of being cached. 084 */ 085 CACHING, 086 087 /** 088 * The MappableBlock was in the process of being cached, but it was 089 * cancelled. Only the FsDatasetCache#WorkerTask can remove cancelled 090 * MappableBlock objects. 091 */ 092 CACHING_CANCELLED, 093 094 /** 095 * The MappableBlock is in the cache. 096 */ 097 CACHED, 098 099 /** 100 * The MappableBlock is in the process of uncaching. 101 */ 102 UNCACHING; 103 104 /** 105 * Whether we should advertise this block as cached to the NameNode and 106 * clients. 107 */ 108 public boolean shouldAdvertise() { 109 return (this == CACHED); 110 } 111 } 112 113 private static final Logger LOG = LoggerFactory.getLogger(FsDatasetCache 114 .class); 115 116 /** 117 * Stores MappableBlock objects and the states they're in. 118 */ 119 private final HashMap<ExtendedBlockId, Value> mappableBlockMap = 120 new HashMap<ExtendedBlockId, Value>(); 121 122 private final AtomicLong numBlocksCached = new AtomicLong(0); 123 124 private final FsDatasetImpl dataset; 125 126 private final ThreadPoolExecutor uncachingExecutor; 127 128 private final ScheduledThreadPoolExecutor deferredUncachingExecutor; 129 130 private final long revocationMs; 131 132 private final long revocationPollingMs; 133 134 /** 135 * The approximate amount of cache space in use. 136 * 137 * This number is an overestimate, counting bytes that will be used only 138 * if pending caching operations succeed. It does not take into account 139 * pending uncaching operations. 140 * 141 * This overestimate is more useful to the NameNode than an underestimate, 142 * since we don't want the NameNode to assign us more replicas than 143 * we can cache, because of the current batch of operations. 144 */ 145 private final UsedBytesCount usedBytesCount; 146 147 public static class PageRounder { 148 private final long osPageSize = 149 NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize(); 150 151 /** 152 * Round up a number to the operating system page size. 153 */ 154 public long roundUp(long count) { 155 return (count + osPageSize - 1) & (~(osPageSize - 1)); 156 } 157 158 /** 159 * Round down a number to the operating system page size. 160 */ 161 public long roundDown(long count) { 162 return count & (~(osPageSize - 1)); 163 } 164 } 165 166 private class UsedBytesCount { 167 private final AtomicLong usedBytes = new AtomicLong(0); 168 169 private final PageRounder rounder = new PageRounder(); 170 171 /** 172 * Try to reserve more bytes. 173 * 174 * @param count The number of bytes to add. We will round this 175 * up to the page size. 176 * 177 * @return The new number of usedBytes if we succeeded; 178 * -1 if we failed. 179 */ 180 long reserve(long count) { 181 count = rounder.roundUp(count); 182 while (true) { 183 long cur = usedBytes.get(); 184 long next = cur + count; 185 if (next > maxBytes) { 186 return -1; 187 } 188 if (usedBytes.compareAndSet(cur, next)) { 189 return next; 190 } 191 } 192 } 193 194 /** 195 * Release some bytes that we're using. 196 * 197 * @param count The number of bytes to release. We will round this 198 * up to the page size. 199 * 200 * @return The new number of usedBytes. 201 */ 202 long release(long count) { 203 count = rounder.roundUp(count); 204 return usedBytes.addAndGet(-count); 205 } 206 207 /** 208 * Release some bytes that we're using rounded down to the page size. 209 * 210 * @param count The number of bytes to release. We will round this 211 * down to the page size. 212 * 213 * @return The new number of usedBytes. 214 */ 215 long releaseRoundDown(long count) { 216 count = rounder.roundDown(count); 217 return usedBytes.addAndGet(-count); 218 } 219 220 long get() { 221 return usedBytes.get(); 222 } 223 } 224 225 /** 226 * The total cache capacity in bytes. 227 */ 228 private final long maxBytes; 229 230 /** 231 * Number of cache commands that could not be completed successfully 232 */ 233 final AtomicLong numBlocksFailedToCache = new AtomicLong(0); 234 /** 235 * Number of uncache commands that could not be completed successfully 236 */ 237 final AtomicLong numBlocksFailedToUncache = new AtomicLong(0); 238 239 public FsDatasetCache(FsDatasetImpl dataset) { 240 this.dataset = dataset; 241 this.maxBytes = dataset.datanode.getDnConf().getMaxLockedMemory(); 242 ThreadFactory workerFactory = new ThreadFactoryBuilder() 243 .setDaemon(true) 244 .setNameFormat("FsDatasetCache-%d-" + dataset.toString()) 245 .build(); 246 this.usedBytesCount = new UsedBytesCount(); 247 this.uncachingExecutor = new ThreadPoolExecutor( 248 0, 1, 249 60, TimeUnit.SECONDS, 250 new LinkedBlockingQueue<Runnable>(), 251 workerFactory); 252 this.uncachingExecutor.allowCoreThreadTimeOut(true); 253 this.deferredUncachingExecutor = new ScheduledThreadPoolExecutor( 254 1, workerFactory); 255 this.revocationMs = dataset.datanode.getConf().getLong( 256 DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS, 257 DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS_DEFAULT); 258 long confRevocationPollingMs = dataset.datanode.getConf().getLong( 259 DFS_DATANODE_CACHE_REVOCATION_POLLING_MS, 260 DFS_DATANODE_CACHE_REVOCATION_POLLING_MS_DEFAULT); 261 long minRevocationPollingMs = revocationMs / 2; 262 if (minRevocationPollingMs < confRevocationPollingMs) { 263 throw new RuntimeException("configured value " + 264 confRevocationPollingMs + "for " + 265 DFS_DATANODE_CACHE_REVOCATION_POLLING_MS + 266 " is too high. It must not be more than half of the " + 267 "value of " + DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS + 268 ". Reconfigure this to " + minRevocationPollingMs); 269 } 270 this.revocationPollingMs = confRevocationPollingMs; 271 } 272 273 /** 274 * @return List of cached blocks suitable for translation into a 275 * {@link BlockListAsLongs} for a cache report. 276 */ 277 synchronized List<Long> getCachedBlocks(String bpid) { 278 List<Long> blocks = new ArrayList<Long>(); 279 for (Iterator<Entry<ExtendedBlockId, Value>> iter = 280 mappableBlockMap.entrySet().iterator(); iter.hasNext(); ) { 281 Entry<ExtendedBlockId, Value> entry = iter.next(); 282 if (entry.getKey().getBlockPoolId().equals(bpid)) { 283 if (entry.getValue().state.shouldAdvertise()) { 284 blocks.add(entry.getKey().getBlockId()); 285 } 286 } 287 } 288 return blocks; 289 } 290 291 /** 292 * Attempt to begin caching a block. 293 */ 294 synchronized void cacheBlock(long blockId, String bpid, 295 String blockFileName, long length, long genstamp, 296 Executor volumeExecutor) { 297 ExtendedBlockId key = new ExtendedBlockId(blockId, bpid); 298 Value prevValue = mappableBlockMap.get(key); 299 if (prevValue != null) { 300 LOG.debug("Block with id {}, pool {} already exists in the " 301 + "FsDatasetCache with state {}", blockId, bpid, prevValue.state 302 ); 303 numBlocksFailedToCache.incrementAndGet(); 304 return; 305 } 306 mappableBlockMap.put(key, new Value(null, State.CACHING)); 307 volumeExecutor.execute( 308 new CachingTask(key, blockFileName, length, genstamp)); 309 LOG.debug("Initiating caching for Block with id {}, pool {}", blockId, 310 bpid); 311 } 312 313 synchronized void uncacheBlock(String bpid, long blockId) { 314 ExtendedBlockId key = new ExtendedBlockId(blockId, bpid); 315 Value prevValue = mappableBlockMap.get(key); 316 boolean deferred = false; 317 318 if (!dataset.datanode.getShortCircuitRegistry(). 319 processBlockMunlockRequest(key)) { 320 deferred = true; 321 } 322 if (prevValue == null) { 323 LOG.debug("Block with id {}, pool {} does not need to be uncached, " 324 + "because it is not currently in the mappableBlockMap.", blockId, 325 bpid); 326 numBlocksFailedToUncache.incrementAndGet(); 327 return; 328 } 329 switch (prevValue.state) { 330 case CACHING: 331 LOG.debug("Cancelling caching for block with id {}, pool {}.", blockId, 332 bpid); 333 mappableBlockMap.put(key, 334 new Value(prevValue.mappableBlock, State.CACHING_CANCELLED)); 335 break; 336 case CACHED: 337 mappableBlockMap.put(key, 338 new Value(prevValue.mappableBlock, State.UNCACHING)); 339 if (deferred) { 340 if (LOG.isDebugEnabled()) { 341 LOG.debug("{} is anchored, and can't be uncached now. Scheduling it " + 342 "for uncaching in {} ", 343 key, DurationFormatUtils.formatDurationHMS(revocationPollingMs)); 344 } 345 deferredUncachingExecutor.schedule( 346 new UncachingTask(key, revocationMs), 347 revocationPollingMs, TimeUnit.MILLISECONDS); 348 } else { 349 LOG.debug("{} has been scheduled for immediate uncaching.", key); 350 uncachingExecutor.execute(new UncachingTask(key, 0)); 351 } 352 break; 353 default: 354 LOG.debug("Block with id {}, pool {} does not need to be uncached, " 355 + "because it is in state {}.", blockId, bpid, prevValue.state); 356 numBlocksFailedToUncache.incrementAndGet(); 357 break; 358 } 359 } 360 361 /** 362 * Try to reserve more bytes. 363 * 364 * @param count The number of bytes to add. We will round this 365 * up to the page size. 366 * 367 * @return The new number of usedBytes if we succeeded; 368 * -1 if we failed. 369 */ 370 long reserve(long count) { 371 return usedBytesCount.reserve(count); 372 } 373 374 /** 375 * Release some bytes that we're using. 376 * 377 * @param count The number of bytes to release. We will round this 378 * up to the page size. 379 * 380 * @return The new number of usedBytes. 381 */ 382 long release(long count) { 383 return usedBytesCount.release(count); 384 } 385 386 /** 387 * Release some bytes that we're using rounded down to the page size. 388 * 389 * @param count The number of bytes to release. We will round this 390 * down to the page size. 391 * 392 * @return The new number of usedBytes. 393 */ 394 long releaseRoundDown(long count) { 395 return usedBytesCount.releaseRoundDown(count); 396 } 397 398 /** 399 * Get the OS page size. 400 * 401 * @return the OS page size. 402 */ 403 long getOsPageSize() { 404 return usedBytesCount.rounder.osPageSize; 405 } 406 407 /** 408 * Round up to the OS page size. 409 */ 410 long roundUpPageSize(long count) { 411 return usedBytesCount.rounder.roundUp(count); 412 } 413 414 /** 415 * Background worker that mmaps, mlocks, and checksums a block 416 */ 417 private class CachingTask implements Runnable { 418 private final ExtendedBlockId key; 419 private final String blockFileName; 420 private final long length; 421 private final long genstamp; 422 423 CachingTask(ExtendedBlockId key, String blockFileName, long length, long genstamp) { 424 this.key = key; 425 this.blockFileName = blockFileName; 426 this.length = length; 427 this.genstamp = genstamp; 428 } 429 430 @Override 431 public void run() { 432 boolean success = false; 433 FileInputStream blockIn = null, metaIn = null; 434 MappableBlock mappableBlock = null; 435 ExtendedBlock extBlk = new ExtendedBlock(key.getBlockPoolId(), 436 key.getBlockId(), length, genstamp); 437 long newUsedBytes = reserve(length); 438 boolean reservedBytes = false; 439 try { 440 if (newUsedBytes < 0) { 441 LOG.warn("Failed to cache " + key + ": could not reserve " + length + 442 " more bytes in the cache: " + 443 DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY + 444 " of " + maxBytes + " exceeded."); 445 return; 446 } 447 reservedBytes = true; 448 try { 449 blockIn = (FileInputStream)dataset.getBlockInputStream(extBlk, 0); 450 metaIn = DatanodeUtil.getMetaDataInputStream(extBlk, dataset); 451 } catch (ClassCastException e) { 452 LOG.warn("Failed to cache " + key + 453 ": Underlying blocks are not backed by files.", e); 454 return; 455 } catch (FileNotFoundException e) { 456 LOG.info("Failed to cache " + key + ": failed to find backing " + 457 "files."); 458 return; 459 } catch (IOException e) { 460 LOG.warn("Failed to cache " + key + ": failed to open file", e); 461 return; 462 } 463 try { 464 mappableBlock = MappableBlock. 465 load(length, blockIn, metaIn, blockFileName); 466 } catch (ChecksumException e) { 467 // Exception message is bogus since this wasn't caused by a file read 468 LOG.warn("Failed to cache " + key + ": checksum verification failed."); 469 return; 470 } catch (IOException e) { 471 LOG.warn("Failed to cache " + key, e); 472 return; 473 } 474 synchronized (FsDatasetCache.this) { 475 Value value = mappableBlockMap.get(key); 476 Preconditions.checkNotNull(value); 477 Preconditions.checkState(value.state == State.CACHING || 478 value.state == State.CACHING_CANCELLED); 479 if (value.state == State.CACHING_CANCELLED) { 480 mappableBlockMap.remove(key); 481 LOG.warn("Caching of " + key + " was cancelled."); 482 return; 483 } 484 mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED)); 485 } 486 LOG.debug("Successfully cached {}. We are now caching {} bytes in" 487 + " total.", key, newUsedBytes); 488 dataset.datanode.getShortCircuitRegistry().processBlockMlockEvent(key); 489 numBlocksCached.addAndGet(1); 490 dataset.datanode.getMetrics().incrBlocksCached(1); 491 success = true; 492 } finally { 493 IOUtils.closeQuietly(blockIn); 494 IOUtils.closeQuietly(metaIn); 495 if (!success) { 496 if (reservedBytes) { 497 release(length); 498 } 499 LOG.debug("Caching of {} was aborted. We are now caching only {} " 500 + "bytes in total.", key, usedBytesCount.get()); 501 if (mappableBlock != null) { 502 mappableBlock.close(); 503 } 504 numBlocksFailedToCache.incrementAndGet(); 505 506 synchronized (FsDatasetCache.this) { 507 mappableBlockMap.remove(key); 508 } 509 } 510 } 511 } 512 } 513 514 private class UncachingTask implements Runnable { 515 private final ExtendedBlockId key; 516 private final long revocationTimeMs; 517 518 UncachingTask(ExtendedBlockId key, long revocationDelayMs) { 519 this.key = key; 520 if (revocationDelayMs == 0) { 521 this.revocationTimeMs = 0; 522 } else { 523 this.revocationTimeMs = revocationDelayMs + Time.monotonicNow(); 524 } 525 } 526 527 private boolean shouldDefer() { 528 /* If revocationTimeMs == 0, this is an immediate uncache request. 529 * No clients were anchored at the time we made the request. */ 530 if (revocationTimeMs == 0) { 531 return false; 532 } 533 /* Let's check if any clients still have this block anchored. */ 534 boolean anchored = 535 !dataset.datanode.getShortCircuitRegistry(). 536 processBlockMunlockRequest(key); 537 if (!anchored) { 538 LOG.debug("Uncaching {} now that it is no longer in use " + 539 "by any clients.", key); 540 return false; 541 } 542 long delta = revocationTimeMs - Time.monotonicNow(); 543 if (delta < 0) { 544 LOG.warn("Forcibly uncaching {} after {} " + 545 "because client(s) {} refused to stop using it.", key, 546 DurationFormatUtils.formatDurationHMS(revocationTimeMs), 547 dataset.datanode.getShortCircuitRegistry().getClientNames(key)); 548 return false; 549 } 550 LOG.info("Replica {} still can't be uncached because some " + 551 "clients continue to use it. Will wait for {}", key, 552 DurationFormatUtils.formatDurationHMS(delta)); 553 return true; 554 } 555 556 @Override 557 public void run() { 558 Value value; 559 560 if (shouldDefer()) { 561 deferredUncachingExecutor.schedule( 562 this, revocationPollingMs, TimeUnit.MILLISECONDS); 563 return; 564 } 565 566 synchronized (FsDatasetCache.this) { 567 value = mappableBlockMap.get(key); 568 } 569 Preconditions.checkNotNull(value); 570 Preconditions.checkArgument(value.state == State.UNCACHING); 571 572 IOUtils.closeQuietly(value.mappableBlock); 573 synchronized (FsDatasetCache.this) { 574 mappableBlockMap.remove(key); 575 } 576 long newUsedBytes = release(value.mappableBlock.getLength()); 577 numBlocksCached.addAndGet(-1); 578 dataset.datanode.getMetrics().incrBlocksUncached(1); 579 if (revocationTimeMs != 0) { 580 LOG.debug("Uncaching of {} completed. usedBytes = {}", 581 key, newUsedBytes); 582 } else { 583 LOG.debug("Deferred uncaching of {} completed. usedBytes = {}", 584 key, newUsedBytes); 585 } 586 } 587 } 588 589 // Stats related methods for FSDatasetMBean 590 591 /** 592 * Get the approximate amount of cache space used. 593 */ 594 public long getCacheUsed() { 595 return usedBytesCount.get(); 596 } 597 598 /** 599 * Get the maximum amount of bytes we can cache. This is a constant. 600 */ 601 public long getCacheCapacity() { 602 return maxBytes; 603 } 604 605 public long getNumBlocksFailedToCache() { 606 return numBlocksFailedToCache.get(); 607 } 608 609 public long getNumBlocksFailedToUncache() { 610 return numBlocksFailedToUncache.get(); 611 } 612 613 public long getNumBlocksCached() { 614 return numBlocksCached.get(); 615 } 616 617 public synchronized boolean isCached(String bpid, long blockId) { 618 ExtendedBlockId block = new ExtendedBlockId(blockId, bpid); 619 Value val = mappableBlockMap.get(block); 620 return (val != null) && val.state.shouldAdvertise(); 621 } 622}