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 round(long count) {
155      long newCount = 
156          (count + (osPageSize - 1)) / osPageSize;
157      return newCount * osPageSize;
158    }
159  }
160
161  private class UsedBytesCount {
162    private final AtomicLong usedBytes = new AtomicLong(0);
163    
164    private final PageRounder rounder = new PageRounder();
165
166    /**
167     * Try to reserve more bytes.
168     *
169     * @param count    The number of bytes to add.  We will round this
170     *                 up to the page size.
171     *
172     * @return         The new number of usedBytes if we succeeded;
173     *                 -1 if we failed.
174     */
175    long reserve(long count) {
176      count = rounder.round(count);
177      while (true) {
178        long cur = usedBytes.get();
179        long next = cur + count;
180        if (next > maxBytes) {
181          return -1;
182        }
183        if (usedBytes.compareAndSet(cur, next)) {
184          return next;
185        }
186      }
187    }
188    
189    /**
190     * Release some bytes that we're using.
191     *
192     * @param count    The number of bytes to release.  We will round this
193     *                 up to the page size.
194     *
195     * @return         The new number of usedBytes.
196     */
197    long release(long count) {
198      count = rounder.round(count);
199      return usedBytes.addAndGet(-count);
200    }
201    
202    long get() {
203      return usedBytes.get();
204    }
205  }
206
207  /**
208   * The total cache capacity in bytes.
209   */
210  private final long maxBytes;
211
212  /**
213   * Number of cache commands that could not be completed successfully
214   */
215  final AtomicLong numBlocksFailedToCache = new AtomicLong(0);
216  /**
217   * Number of uncache commands that could not be completed successfully
218   */
219  final AtomicLong numBlocksFailedToUncache = new AtomicLong(0);
220
221  public FsDatasetCache(FsDatasetImpl dataset) {
222    this.dataset = dataset;
223    this.maxBytes = dataset.datanode.getDnConf().getMaxLockedMemory();
224    ThreadFactory workerFactory = new ThreadFactoryBuilder()
225        .setDaemon(true)
226        .setNameFormat("FsDatasetCache-%d-" + dataset.toString())
227        .build();
228    this.usedBytesCount = new UsedBytesCount();
229    this.uncachingExecutor = new ThreadPoolExecutor(
230            0, 1,
231            60, TimeUnit.SECONDS,
232            new LinkedBlockingQueue<Runnable>(),
233            workerFactory);
234    this.uncachingExecutor.allowCoreThreadTimeOut(true);
235    this.deferredUncachingExecutor = new ScheduledThreadPoolExecutor(
236            1, workerFactory);
237    this.revocationMs = dataset.datanode.getConf().getLong(
238        DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS,
239        DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS_DEFAULT);
240    long confRevocationPollingMs = dataset.datanode.getConf().getLong(
241        DFS_DATANODE_CACHE_REVOCATION_POLLING_MS,
242        DFS_DATANODE_CACHE_REVOCATION_POLLING_MS_DEFAULT);
243    long minRevocationPollingMs = revocationMs / 2;
244    if (minRevocationPollingMs < confRevocationPollingMs) {
245      throw new RuntimeException("configured value " +
246              confRevocationPollingMs + "for " +
247              DFS_DATANODE_CACHE_REVOCATION_POLLING_MS +
248              " is too high.  It must not be more than half of the " +
249              "value of " +  DFS_DATANODE_CACHE_REVOCATION_TIMEOUT_MS +
250              ".  Reconfigure this to " + minRevocationPollingMs);
251    }
252    this.revocationPollingMs = confRevocationPollingMs;
253  }
254
255  /**
256   * @return List of cached blocks suitable for translation into a
257   * {@link BlockListAsLongs} for a cache report.
258   */
259  synchronized List<Long> getCachedBlocks(String bpid) {
260    List<Long> blocks = new ArrayList<Long>();
261    for (Iterator<Entry<ExtendedBlockId, Value>> iter =
262        mappableBlockMap.entrySet().iterator(); iter.hasNext(); ) {
263      Entry<ExtendedBlockId, Value> entry = iter.next();
264      if (entry.getKey().getBlockPoolId().equals(bpid)) {
265        if (entry.getValue().state.shouldAdvertise()) {
266          blocks.add(entry.getKey().getBlockId());
267        }
268      }
269    }
270    return blocks;
271  }
272
273  /**
274   * Attempt to begin caching a block.
275   */
276  synchronized void cacheBlock(long blockId, String bpid,
277      String blockFileName, long length, long genstamp,
278      Executor volumeExecutor) {
279    ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
280    Value prevValue = mappableBlockMap.get(key);
281    if (prevValue != null) {
282      LOG.debug("Block with id {}, pool {} already exists in the "
283              + "FsDatasetCache with state {}", blockId, bpid, prevValue.state
284      );
285      numBlocksFailedToCache.incrementAndGet();
286      return;
287    }
288    mappableBlockMap.put(key, new Value(null, State.CACHING));
289    volumeExecutor.execute(
290        new CachingTask(key, blockFileName, length, genstamp));
291    LOG.debug("Initiating caching for Block with id {}, pool {}", blockId,
292        bpid);
293  }
294
295  synchronized void uncacheBlock(String bpid, long blockId) {
296    ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
297    Value prevValue = mappableBlockMap.get(key);
298    boolean deferred = false;
299
300    if (!dataset.datanode.getShortCircuitRegistry().
301            processBlockMunlockRequest(key)) {
302      deferred = true;
303    }
304    if (prevValue == null) {
305      LOG.debug("Block with id {}, pool {} does not need to be uncached, "
306          + "because it is not currently in the mappableBlockMap.", blockId,
307          bpid);
308      numBlocksFailedToUncache.incrementAndGet();
309      return;
310    }
311    switch (prevValue.state) {
312    case CACHING:
313      LOG.debug("Cancelling caching for block with id {}, pool {}.", blockId,
314          bpid);
315      mappableBlockMap.put(key,
316          new Value(prevValue.mappableBlock, State.CACHING_CANCELLED));
317      break;
318    case CACHED:
319      mappableBlockMap.put(key,
320          new Value(prevValue.mappableBlock, State.UNCACHING));
321      if (deferred) {
322        LOG.debug("{} is anchored, and can't be uncached now.  Scheduling it " +
323            "for uncaching in {} ",
324            key, DurationFormatUtils.formatDurationHMS(revocationPollingMs));
325        deferredUncachingExecutor.schedule(
326            new UncachingTask(key, revocationMs),
327            revocationPollingMs, TimeUnit.MILLISECONDS);
328      } else {
329        LOG.debug("{} has been scheduled for immediate uncaching.", key);
330        uncachingExecutor.execute(new UncachingTask(key, 0));
331      }
332      break;
333    default:
334      LOG.debug("Block with id {}, pool {} does not need to be uncached, "
335          + "because it is in state {}.", blockId, bpid, prevValue.state);
336      numBlocksFailedToUncache.incrementAndGet();
337      break;
338    }
339  }
340
341  /**
342   * Background worker that mmaps, mlocks, and checksums a block
343   */
344  private class CachingTask implements Runnable {
345    private final ExtendedBlockId key; 
346    private final String blockFileName;
347    private final long length;
348    private final long genstamp;
349
350    CachingTask(ExtendedBlockId key, String blockFileName, long length, long genstamp) {
351      this.key = key;
352      this.blockFileName = blockFileName;
353      this.length = length;
354      this.genstamp = genstamp;
355    }
356
357    @Override
358    public void run() {
359      boolean success = false;
360      FileInputStream blockIn = null, metaIn = null;
361      MappableBlock mappableBlock = null;
362      ExtendedBlock extBlk = new ExtendedBlock(key.getBlockPoolId(),
363          key.getBlockId(), length, genstamp);
364      long newUsedBytes = usedBytesCount.reserve(length);
365      boolean reservedBytes = false;
366      try {
367        if (newUsedBytes < 0) {
368          LOG.warn("Failed to cache " + key + ": could not reserve " + length +
369              " more bytes in the cache: " +
370              DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY +
371              " of " + maxBytes + " exceeded.");
372          return;
373        }
374        reservedBytes = true;
375        try {
376          blockIn = (FileInputStream)dataset.getBlockInputStream(extBlk, 0);
377          metaIn = DatanodeUtil.getMetaDataInputStream(extBlk, dataset);
378        } catch (ClassCastException e) {
379          LOG.warn("Failed to cache " + key +
380              ": Underlying blocks are not backed by files.", e);
381          return;
382        } catch (FileNotFoundException e) {
383          LOG.info("Failed to cache " + key + ": failed to find backing " +
384              "files.");
385          return;
386        } catch (IOException e) {
387          LOG.warn("Failed to cache " + key + ": failed to open file", e);
388          return;
389        }
390        try {
391          mappableBlock = MappableBlock.
392              load(length, blockIn, metaIn, blockFileName);
393        } catch (ChecksumException e) {
394          // Exception message is bogus since this wasn't caused by a file read
395          LOG.warn("Failed to cache " + key + ": checksum verification failed.");
396          return;
397        } catch (IOException e) {
398          LOG.warn("Failed to cache " + key, e);
399          return;
400        }
401        synchronized (FsDatasetCache.this) {
402          Value value = mappableBlockMap.get(key);
403          Preconditions.checkNotNull(value);
404          Preconditions.checkState(value.state == State.CACHING ||
405                                   value.state == State.CACHING_CANCELLED);
406          if (value.state == State.CACHING_CANCELLED) {
407            mappableBlockMap.remove(key);
408            LOG.warn("Caching of " + key + " was cancelled.");
409            return;
410          }
411          mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED));
412        }
413        LOG.debug("Successfully cached {}.  We are now caching {} bytes in"
414            + " total.", key, newUsedBytes);
415        dataset.datanode.getShortCircuitRegistry().processBlockMlockEvent(key);
416        numBlocksCached.addAndGet(1);
417        dataset.datanode.getMetrics().incrBlocksCached(1);
418        success = true;
419      } finally {
420        IOUtils.closeQuietly(blockIn);
421        IOUtils.closeQuietly(metaIn);
422        if (!success) {
423          if (reservedBytes) {
424            usedBytesCount.release(length);
425          }
426          LOG.debug("Caching of {} was aborted.  We are now caching only {} "
427                  + "bytes in total.", key, usedBytesCount.get());
428          if (mappableBlock != null) {
429            mappableBlock.close();
430          }
431          numBlocksFailedToCache.incrementAndGet();
432
433          synchronized (FsDatasetCache.this) {
434            mappableBlockMap.remove(key);
435          }
436        }
437      }
438    }
439  }
440
441  private class UncachingTask implements Runnable {
442    private final ExtendedBlockId key; 
443    private final long revocationTimeMs;
444
445    UncachingTask(ExtendedBlockId key, long revocationDelayMs) {
446      this.key = key;
447      if (revocationDelayMs == 0) {
448        this.revocationTimeMs = 0;
449      } else {
450        this.revocationTimeMs = revocationDelayMs + Time.monotonicNow();
451      }
452    }
453
454    private boolean shouldDefer() {
455      /* If revocationTimeMs == 0, this is an immediate uncache request.
456       * No clients were anchored at the time we made the request. */
457      if (revocationTimeMs == 0) {
458        return false;
459      }
460      /* Let's check if any clients still have this block anchored. */
461      boolean anchored =
462        !dataset.datanode.getShortCircuitRegistry().
463            processBlockMunlockRequest(key);
464      if (!anchored) {
465        LOG.debug("Uncaching {} now that it is no longer in use " +
466            "by any clients.", key);
467        return false;
468      }
469      long delta = revocationTimeMs - Time.monotonicNow();
470      if (delta < 0) {
471        LOG.warn("Forcibly uncaching {} after {} " +
472            "because client(s) {} refused to stop using it.", key,
473            DurationFormatUtils.formatDurationHMS(revocationTimeMs),
474            dataset.datanode.getShortCircuitRegistry().getClientNames(key));
475        return false;
476      }
477      LOG.info("Replica {} still can't be uncached because some " +
478          "clients continue to use it.  Will wait for {}", key,
479          DurationFormatUtils.formatDurationHMS(delta));
480      return true;
481    }
482
483    @Override
484    public void run() {
485      Value value;
486
487      if (shouldDefer()) {
488        deferredUncachingExecutor.schedule(
489            this, revocationPollingMs, TimeUnit.MILLISECONDS);
490        return;
491      }
492
493      synchronized (FsDatasetCache.this) {
494        value = mappableBlockMap.get(key);
495      }
496      Preconditions.checkNotNull(value);
497      Preconditions.checkArgument(value.state == State.UNCACHING);
498
499      IOUtils.closeQuietly(value.mappableBlock);
500      synchronized (FsDatasetCache.this) {
501        mappableBlockMap.remove(key);
502      }
503      long newUsedBytes =
504          usedBytesCount.release(value.mappableBlock.getLength());
505      numBlocksCached.addAndGet(-1);
506      dataset.datanode.getMetrics().incrBlocksUncached(1);
507      if (revocationTimeMs != 0) {
508        LOG.debug("Uncaching of {} completed. usedBytes = {}",
509            key, newUsedBytes);
510      } else {
511        LOG.debug("Deferred uncaching of {} completed. usedBytes = {}",
512            key, newUsedBytes);
513      }
514    }
515  }
516
517  // Stats related methods for FSDatasetMBean
518
519  /**
520   * Get the approximate amount of cache space used.
521   */
522  public long getCacheUsed() {
523    return usedBytesCount.get();
524  }
525
526  /**
527   * Get the maximum amount of bytes we can cache.  This is a constant.
528   */
529  public long getCacheCapacity() {
530    return maxBytes;
531  }
532
533  public long getNumBlocksFailedToCache() {
534    return numBlocksFailedToCache.get();
535  }
536
537  public long getNumBlocksFailedToUncache() {
538    return numBlocksFailedToUncache.get();
539  }
540
541  public long getNumBlocksCached() {
542    return numBlocksCached.get();
543  }
544
545  public synchronized boolean isCached(String bpid, long blockId) {
546    ExtendedBlockId block = new ExtendedBlockId(blockId, bpid);
547    Value val = mappableBlockMap.get(block);
548    return (val != null) && val.state.shouldAdvertise();
549  }
550}