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}