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.hdfs.client;
019
020import java.io.Closeable;
021
022import org.apache.hadoop.classification.InterfaceAudience;
023
024import java.io.FileInputStream;
025import java.io.IOException;
026import java.lang.ref.WeakReference;
027import java.util.Iterator;
028import java.util.TreeMap;
029import java.util.Map.Entry;
030import java.util.concurrent.ScheduledFuture;
031import java.util.concurrent.ScheduledThreadPoolExecutor;
032import java.util.concurrent.TimeUnit;
033import java.util.concurrent.locks.Condition;
034import java.util.concurrent.locks.Lock;
035import java.util.concurrent.locks.ReentrantLock;
036
037import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE;
038import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT;
039import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS;
040import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT;
041import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT;
042import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT;
043
044import org.apache.commons.logging.Log;
045import org.apache.commons.logging.LogFactory;
046import org.apache.hadoop.conf.Configuration;
047import org.apache.hadoop.hdfs.protocol.DatanodeID;
048import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
049import org.apache.hadoop.io.IOUtils;
050
051import com.google.common.annotations.VisibleForTesting;
052import com.google.common.collect.ComparisonChain;
053import com.google.common.util.concurrent.ThreadFactoryBuilder;
054
055/**
056 * Tracks mmap instances used on an HDFS client.
057 *
058 * mmaps can be used concurrently by multiple threads at once.
059 * mmaps cannot be closed while they are in use.
060 *
061 * The cache is important for performance, because the first time an mmap is
062 * created, the page table entries (PTEs) are not yet set up.
063 * Even when reading data that is entirely resident in memory, reading an
064 * mmap the second time is faster.
065 */
066@InterfaceAudience.Private
067public class ClientMmapManager implements Closeable {
068  public static final Log LOG = LogFactory.getLog(ClientMmapManager.class);
069
070  private boolean closed = false;
071
072  private final int cacheSize;
073
074  private final long timeoutNs;
075
076  private final int runsPerTimeout;
077
078  private final Lock lock = new ReentrantLock();
079  
080  /**
081   * Maps block, datanode_id to the client mmap object.
082   * If the ClientMmap is in the process of being loaded,
083   * {@link Waitable<ClientMmap>#await()} will block.
084   *
085   * Protected by the ClientMmapManager lock.
086   */
087  private final TreeMap<Key, Waitable<ClientMmap>> mmaps =
088      new TreeMap<Key, Waitable<ClientMmap>>();
089
090  /**
091   * Maps the last use time to the client mmap object.
092   * We ensure that each last use time is unique by inserting a jitter of a
093   * nanosecond or two if necessary.
094   * 
095   * Protected by the ClientMmapManager lock.
096   * ClientMmap objects that are in use are never evictable.
097   */
098  private final TreeMap<Long, ClientMmap> evictable =
099      new TreeMap<Long, ClientMmap>();
100
101  private final ScheduledThreadPoolExecutor executor = 
102      new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
103          setDaemon(true).setNameFormat("ClientMmapManager").
104          build());
105  
106  /**
107   * The CacheCleaner for this ClientMmapManager.  We don't create this
108   * and schedule it until it becomes necessary.
109   */
110  private CacheCleaner cacheCleaner;
111
112  /**
113   * Factory method to create a ClientMmapManager from a Hadoop
114   * configuration.
115   */
116  public static ClientMmapManager fromConf(Configuration conf) {
117    return new ClientMmapManager(conf.getInt(DFS_CLIENT_MMAP_CACHE_SIZE,
118      DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT),
119      conf.getLong(DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
120        DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT),
121      conf.getInt(DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT,
122        DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT));
123  }
124
125  public ClientMmapManager(int cacheSize, long timeoutMs, int runsPerTimeout) {
126    this.cacheSize = cacheSize;
127    this.timeoutNs = timeoutMs * 1000000;
128    this.runsPerTimeout = runsPerTimeout;
129  }
130  
131  long getTimeoutMs() {
132    return this.timeoutNs / 1000000;
133  }
134
135  int getRunsPerTimeout() {
136    return this.runsPerTimeout;
137  }
138  
139  public String verifyConfigurationMatches(Configuration conf) {
140    StringBuilder bld = new StringBuilder();
141    int cacheSize = conf.getInt(DFS_CLIENT_MMAP_CACHE_SIZE,
142                    DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
143    if (this.cacheSize != cacheSize) {
144      bld.append("You specified a cache size of ").append(cacheSize).
145          append(", but the existing cache size is ").append(this.cacheSize).
146          append(".  ");
147    }
148    long timeoutMs = conf.getLong(DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
149        DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
150    if (getTimeoutMs() != timeoutMs) {
151      bld.append("You specified a cache timeout of ").append(timeoutMs).
152          append(" ms, but the existing cache timeout is ").
153          append(getTimeoutMs()).append("ms").append(".  ");
154    }
155    int runsPerTimeout = conf.getInt(
156        DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT,
157        DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT);
158    if (getRunsPerTimeout() != runsPerTimeout) {
159      bld.append("You specified ").append(runsPerTimeout).
160          append(" runs per timeout, but the existing runs per timeout is ").
161          append(getTimeoutMs()).append(".  ");
162    }
163    return bld.toString();
164  }
165
166  private static class Waitable<T> {
167    private T val;
168    private final Condition cond;
169
170    public Waitable(Condition cond) {
171      this.val = null;
172      this.cond = cond;
173    }
174
175    public T await() throws InterruptedException {
176      while (this.val == null) {
177        this.cond.await();
178      }
179      return this.val;
180    }
181
182    public void provide(T val) {
183      this.val = val;
184      this.cond.signalAll();
185    }
186  }
187
188  private static class Key implements Comparable<Key> {
189    private final ExtendedBlock block;
190    private final DatanodeID datanode;
191    
192    Key(ExtendedBlock block, DatanodeID datanode) {
193      this.block = block;
194      this.datanode = datanode;
195    }
196
197    /**
198     * Compare two ClientMmap regions that we're storing.
199     *
200     * When we append to a block, we bump the genstamp.  It is important to 
201     * compare the genStamp here.  That way, we will not return a shorter 
202     * mmap than required.
203     */
204    @Override
205    public int compareTo(Key o) {
206      return ComparisonChain.start().
207          compare(block.getBlockId(), o.block.getBlockId()).
208          compare(block.getGenerationStamp(), o.block.getGenerationStamp()).
209          compare(block.getBlockPoolId(), o.block.getBlockPoolId()).
210          compare(datanode, o.datanode).
211          result();
212    }
213
214    @Override
215    public boolean equals(Object rhs) {
216      if (rhs == null) {
217        return false;
218      }
219      try {
220        Key o = (Key)rhs;
221        return (compareTo(o) == 0);
222      } catch (ClassCastException e) {
223        return false;
224      }
225    }
226
227    @Override
228    public int hashCode() {
229      return block.hashCode() ^ datanode.hashCode();
230    }
231  }
232
233  /**
234   * Thread which handles expiring mmaps from the cache.
235   */
236  private static class CacheCleaner implements Runnable, Closeable {
237    private WeakReference<ClientMmapManager> managerRef;
238    private ScheduledFuture<?> future;
239    
240    CacheCleaner(ClientMmapManager manager) {
241      this.managerRef= new WeakReference<ClientMmapManager>(manager);
242    }
243
244    @Override
245    public void run() {
246      ClientMmapManager manager = managerRef.get();
247      if (manager == null) return;
248      long curTime = System.nanoTime();
249      try {
250        manager.lock.lock();
251        manager.evictStaleEntries(curTime);
252      } finally {
253        manager.lock.unlock();
254      }
255    }
256    
257    void setFuture(ScheduledFuture<?> future) {
258      this.future = future;
259    }
260
261    @Override
262    public void close() throws IOException {
263      future.cancel(false);
264    }
265  }
266
267  /**
268   * Evict entries which are older than curTime + timeoutNs from the cache.
269   *
270   * NOTE: you must call this function with the lock held.
271   */
272  private void evictStaleEntries(long curTime) {
273    if (closed) {
274      return;
275    }
276    Iterator<Entry<Long, ClientMmap>> iter =
277        evictable.entrySet().iterator(); 
278    while (iter.hasNext()) {
279      Entry<Long, ClientMmap> entry = iter.next();
280      if (entry.getKey() + timeoutNs >= curTime) {
281        return;
282      }
283      ClientMmap mmap = entry.getValue();
284      Key key = new Key(mmap.getBlock(), mmap.getDatanodeID());
285      mmaps.remove(key);
286      iter.remove();
287      mmap.unmap();
288    }
289  }
290
291  /**
292   * Evict one mmap object from the cache.
293   *
294   * NOTE: you must call this function with the lock held.
295   *
296   * @return                  True if an object was evicted; false if none
297   *                          could be evicted.
298   */
299  private boolean evictOne() {
300    Entry<Long, ClientMmap> entry = evictable.pollFirstEntry();
301    if (entry == null) {
302      // We don't want to try creating another mmap region, because the
303      // cache is full.
304      return false;
305    }
306    ClientMmap evictedMmap = entry.getValue(); 
307    Key evictedKey = new Key(evictedMmap.getBlock(), 
308                             evictedMmap.getDatanodeID());
309    mmaps.remove(evictedKey);
310    evictedMmap.unmap();
311    return true;
312  }
313
314  /**
315   * Create a new mmap object.
316   * 
317   * NOTE: you must call this function with the lock held.
318   *
319   * @param key              The key which describes this mmap.
320   * @param in               The input stream to use to create the mmap.
321   * @return                 The new mmap object, or null if there were
322   *                         insufficient resources.
323   * @throws IOException     If there was an I/O error creating the mmap.
324   */
325  private ClientMmap create(Key key, FileInputStream in) throws IOException {
326    if (mmaps.size() + 1 > cacheSize) {
327      if (!evictOne()) {
328        LOG.warn("mmap cache is full (with " + cacheSize + " elements) and " +
329              "nothing is evictable.  Ignoring request for mmap with " +
330              "datanodeID=" + key.datanode + ", " + "block=" + key.block);
331        return null;
332      }
333    }
334    // Create the condition variable that other threads may wait on.
335    Waitable<ClientMmap> waitable =
336        new Waitable<ClientMmap>(lock.newCondition());
337    mmaps.put(key, waitable);
338    // Load the entry
339    boolean success = false;
340    ClientMmap mmap = null;
341    try {
342      try {
343        lock.unlock();
344        mmap = ClientMmap.load(this, in, key.block, key.datanode);
345      } finally {
346        lock.lock();
347      }
348      if (cacheCleaner == null) {
349        cacheCleaner = new CacheCleaner(this);
350        ScheduledFuture<?> future = 
351            executor.scheduleAtFixedRate(cacheCleaner,
352                timeoutNs, timeoutNs / runsPerTimeout, TimeUnit.NANOSECONDS);
353        cacheCleaner.setFuture(future);
354      }
355      success = true;
356    } finally {
357      if (!success) {
358        LOG.warn("failed to create mmap for datanodeID=" + key.datanode +
359                  ", " + "block=" + key.block);
360        mmaps.remove(key);
361      }
362      waitable.provide(mmap);
363    }
364    if (LOG.isDebugEnabled()) {
365      LOG.info("created a new ClientMmap for block " + key.block +
366          " on datanode " + key.datanode);
367    }
368    return mmap;
369  }
370
371  /**
372   * Get or create an mmap region.
373   * 
374   * @param node       The DataNode that owns the block for this mmap region.
375   * @param block      The block ID, block pool ID, and generation stamp of 
376   *                     the block we want to read.
377   * @param in         An open file for this block.  This stream is only used
378   *                     if we have to create a new mmap; if we use an
379   *                     existing one, it is ignored.
380   *
381   * @return           The client mmap region.
382   */
383  public ClientMmap fetch(DatanodeID datanodeID, ExtendedBlock block,
384      FileInputStream in) throws IOException, InterruptedException {
385    LOG.debug("fetching mmap with datanodeID=" + datanodeID + ", " +
386        "block=" + block);
387    Key key = new Key(block, datanodeID);
388    ClientMmap mmap = null;
389    try {
390      lock.lock();
391      if (closed) {
392        throw new IOException("ClientMmapManager is closed.");
393      }
394      while (mmap == null) {
395        Waitable<ClientMmap> entry = mmaps.get(key);
396        if (entry == null) {
397          return create(key, in);
398        }
399        mmap = entry.await();
400      }
401      if (mmap.ref() == 1) {
402        // When going from nobody using the mmap (ref = 0) to somebody
403        // using the mmap (ref = 1), we must make the mmap un-evictable.
404        evictable.remove(mmap.getLastEvictableTimeNs());
405      }
406    }
407    finally {
408      lock.unlock();
409    }
410    if (LOG.isDebugEnabled()) {
411      LOG.debug("reusing existing mmap with datanodeID=" + datanodeID +
412              ", " + "block=" + block);
413    }
414    return mmap;
415  }
416
417  /**
418   * Make an mmap evictable.
419   * 
420   * When an mmap is evictable, it may be removed from the cache if necessary.
421   * mmaps can only be evictable if nobody is using them.
422   *
423   * @param mmap             The mmap to make evictable.
424   */
425  void makeEvictable(ClientMmap mmap) {
426    try {
427      lock.lock();
428      if (closed) {
429        // If this ClientMmapManager is closed, then don't bother with the
430        // cache; just close the mmap.
431        mmap.unmap();
432        return;
433      }
434      long now = System.nanoTime();
435      while (evictable.containsKey(now)) {
436        now++;
437      }
438      mmap.setLastEvictableTimeNs(now);
439      evictable.put(now, mmap);
440    } finally {
441      lock.unlock();
442    }
443  }
444
445  @Override
446  public void close() throws IOException {
447    try {
448      lock.lock();
449      closed = true;
450      IOUtils.cleanup(LOG, cacheCleaner);
451
452      // Unmap all the mmaps that nobody is using.
453      // The ones which are in use will be unmapped just as soon as people stop
454      // using them.
455      evictStaleEntries(Long.MAX_VALUE);
456
457      executor.shutdown();
458    } finally {
459      lock.unlock();
460    }
461  }
462
463  @VisibleForTesting
464  public interface ClientMmapVisitor {
465    void accept(ClientMmap mmap);
466  }
467
468  @VisibleForTesting
469  public synchronized void visitMmaps(ClientMmapVisitor visitor)
470      throws InterruptedException {
471    for (Waitable<ClientMmap> entry : mmaps.values()) {
472      visitor.accept(entry.await());
473    }
474  }
475
476  public void visitEvictable(ClientMmapVisitor visitor)
477      throws InterruptedException {
478    for (ClientMmap mmap : evictable.values()) {
479      visitor.accept(mmap);
480    }
481  }
482}