|
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
|
|
|
-import org.apache.hadoop.io.nativeio.NativeIO;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -131,104 +130,10 @@ public class FsDatasetCache {
|
|
|
|
|
|
private final long revocationPollingMs;
|
|
|
|
|
|
- /**
|
|
|
- * The approximate amount of cache space in use.
|
|
|
- *
|
|
|
- * This number is an overestimate, counting bytes that will be used only
|
|
|
- * if pending caching operations succeed. It does not take into account
|
|
|
- * pending uncaching operations.
|
|
|
- *
|
|
|
- * This overestimate is more useful to the NameNode than an underestimate,
|
|
|
- * since we don't want the NameNode to assign us more replicas than
|
|
|
- * we can cache, because of the current batch of operations.
|
|
|
- */
|
|
|
- private final UsedBytesCount usedBytesCount;
|
|
|
-
|
|
|
- public static class PageRounder {
|
|
|
- private final long osPageSize =
|
|
|
- NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
|
|
|
-
|
|
|
- /**
|
|
|
- * Round up a number to the operating system page size.
|
|
|
- */
|
|
|
- public long roundUp(long count) {
|
|
|
- return (count + osPageSize - 1) & (~(osPageSize - 1));
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Round down a number to the operating system page size.
|
|
|
- */
|
|
|
- public long roundDown(long count) {
|
|
|
- return count & (~(osPageSize - 1));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private class UsedBytesCount {
|
|
|
- private final AtomicLong usedBytes = new AtomicLong(0);
|
|
|
-
|
|
|
- private final PageRounder rounder = new PageRounder();
|
|
|
-
|
|
|
- /**
|
|
|
- * Try to reserve more bytes.
|
|
|
- *
|
|
|
- * @param count The number of bytes to add. We will round this
|
|
|
- * up to the page size.
|
|
|
- *
|
|
|
- * @return The new number of usedBytes if we succeeded;
|
|
|
- * -1 if we failed.
|
|
|
- */
|
|
|
- long reserve(long count) {
|
|
|
- count = rounder.roundUp(count);
|
|
|
- while (true) {
|
|
|
- long cur = usedBytes.get();
|
|
|
- long next = cur + count;
|
|
|
- if (next > maxBytes) {
|
|
|
- return -1;
|
|
|
- }
|
|
|
- if (usedBytes.compareAndSet(cur, next)) {
|
|
|
- return next;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Release some bytes that we're using.
|
|
|
- *
|
|
|
- * @param count The number of bytes to release. We will round this
|
|
|
- * up to the page size.
|
|
|
- *
|
|
|
- * @return The new number of usedBytes.
|
|
|
- */
|
|
|
- long release(long count) {
|
|
|
- count = rounder.roundUp(count);
|
|
|
- return usedBytes.addAndGet(-count);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Release some bytes that we're using rounded down to the page size.
|
|
|
- *
|
|
|
- * @param count The number of bytes to release. We will round this
|
|
|
- * down to the page size.
|
|
|
- *
|
|
|
- * @return The new number of usedBytes.
|
|
|
- */
|
|
|
- long releaseRoundDown(long count) {
|
|
|
- count = rounder.roundDown(count);
|
|
|
- return usedBytes.addAndGet(-count);
|
|
|
- }
|
|
|
-
|
|
|
- long get() {
|
|
|
- return usedBytes.get();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * The total cache capacity in bytes.
|
|
|
- */
|
|
|
- private final long maxBytes;
|
|
|
-
|
|
|
private final MappableBlockLoader mappableBlockLoader;
|
|
|
|
|
|
+ private final MemoryCacheStats memCacheStats;
|
|
|
+
|
|
|
/**
|
|
|
* Number of cache commands that could not be completed successfully
|
|
|
*/
|
|
@@ -240,12 +145,10 @@ public class FsDatasetCache {
|
|
|
|
|
|
public FsDatasetCache(FsDatasetImpl dataset) throws IOException {
|
|
|
this.dataset = dataset;
|
|
|
- this.maxBytes = dataset.datanode.getDnConf().getMaxLockedMemory();
|
|
|
ThreadFactory workerFactory = new ThreadFactoryBuilder()
|
|
|
.setDaemon(true)
|
|
|
.setNameFormat("FsDatasetCache-%d-" + dataset.toString())
|
|
|
.build();
|
|
|
- this.usedBytesCount = new UsedBytesCount();
|
|
|
this.uncachingExecutor = new ThreadPoolExecutor(
|
|
|
0, 1,
|
|
|
60, TimeUnit.SECONDS,
|
|
@@ -270,7 +173,11 @@ public class FsDatasetCache {
|
|
|
". Reconfigure this to " + minRevocationPollingMs);
|
|
|
}
|
|
|
this.revocationPollingMs = confRevocationPollingMs;
|
|
|
- this.mappableBlockLoader = new MemoryMappableBlockLoader();
|
|
|
+
|
|
|
+ this.mappableBlockLoader = new MemoryMappableBlockLoader(this);
|
|
|
+ // Both lazy writer and read cache are sharing this statistics.
|
|
|
+ this.memCacheStats = new MemoryCacheStats(
|
|
|
+ dataset.datanode.getDnConf().getMaxLockedMemory());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -371,7 +278,7 @@ public class FsDatasetCache {
|
|
|
* -1 if we failed.
|
|
|
*/
|
|
|
long reserve(long count) {
|
|
|
- return usedBytesCount.reserve(count);
|
|
|
+ return memCacheStats.reserve(count);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -383,7 +290,7 @@ public class FsDatasetCache {
|
|
|
* @return The new number of usedBytes.
|
|
|
*/
|
|
|
long release(long count) {
|
|
|
- return usedBytesCount.release(count);
|
|
|
+ return memCacheStats.release(count);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -395,7 +302,7 @@ public class FsDatasetCache {
|
|
|
* @return The new number of usedBytes.
|
|
|
*/
|
|
|
long releaseRoundDown(long count) {
|
|
|
- return usedBytesCount.releaseRoundDown(count);
|
|
|
+ return memCacheStats.releaseRoundDown(count);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -404,14 +311,14 @@ public class FsDatasetCache {
|
|
|
* @return the OS page size.
|
|
|
*/
|
|
|
long getOsPageSize() {
|
|
|
- return usedBytesCount.rounder.osPageSize;
|
|
|
+ return memCacheStats.getPageSize();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Round up to the OS page size.
|
|
|
*/
|
|
|
long roundUpPageSize(long count) {
|
|
|
- return usedBytesCount.rounder.roundUp(count);
|
|
|
+ return memCacheStats.roundUpPageSize(count);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -437,14 +344,14 @@ public class FsDatasetCache {
|
|
|
MappableBlock mappableBlock = null;
|
|
|
ExtendedBlock extBlk = new ExtendedBlock(key.getBlockPoolId(),
|
|
|
key.getBlockId(), length, genstamp);
|
|
|
- long newUsedBytes = reserve(length);
|
|
|
+ long newUsedBytes = mappableBlockLoader.reserve(length);
|
|
|
boolean reservedBytes = false;
|
|
|
try {
|
|
|
if (newUsedBytes < 0) {
|
|
|
LOG.warn("Failed to cache " + key + ": could not reserve " + length +
|
|
|
" more bytes in the cache: " +
|
|
|
DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY +
|
|
|
- " of " + maxBytes + " exceeded.");
|
|
|
+ " of " + memCacheStats.getCacheCapacity() + " exceeded.");
|
|
|
return;
|
|
|
}
|
|
|
reservedBytes = true;
|
|
@@ -497,10 +404,10 @@ public class FsDatasetCache {
|
|
|
IOUtils.closeQuietly(metaIn);
|
|
|
if (!success) {
|
|
|
if (reservedBytes) {
|
|
|
- release(length);
|
|
|
+ mappableBlockLoader.release(length);
|
|
|
}
|
|
|
LOG.debug("Caching of {} was aborted. We are now caching only {} "
|
|
|
- + "bytes in total.", key, usedBytesCount.get());
|
|
|
+ + "bytes in total.", key, memCacheStats.getCacheUsed());
|
|
|
IOUtils.closeQuietly(mappableBlock);
|
|
|
numBlocksFailedToCache.incrementAndGet();
|
|
|
|
|
@@ -574,7 +481,8 @@ public class FsDatasetCache {
|
|
|
synchronized (FsDatasetCache.this) {
|
|
|
mappableBlockMap.remove(key);
|
|
|
}
|
|
|
- long newUsedBytes = release(value.mappableBlock.getLength());
|
|
|
+ long newUsedBytes = mappableBlockLoader
|
|
|
+ .release(value.mappableBlock.getLength());
|
|
|
numBlocksCached.addAndGet(-1);
|
|
|
dataset.datanode.getMetrics().incrBlocksUncached(1);
|
|
|
if (revocationTimeMs != 0) {
|
|
@@ -593,14 +501,14 @@ public class FsDatasetCache {
|
|
|
* Get the approximate amount of cache space used.
|
|
|
*/
|
|
|
public long getCacheUsed() {
|
|
|
- return usedBytesCount.get();
|
|
|
+ return memCacheStats.getCacheUsed();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Get the maximum amount of bytes we can cache. This is a constant.
|
|
|
*/
|
|
|
public long getCacheCapacity() {
|
|
|
- return maxBytes;
|
|
|
+ return memCacheStats.getCacheCapacity();
|
|
|
}
|
|
|
|
|
|
public long getNumBlocksFailedToCache() {
|