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