|
@@ -17,8 +17,8 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES;
|
|
@@ -40,6 +40,7 @@ import java.util.List;
|
|
|
import java.util.Map.Entry;
|
|
|
import java.util.SortedMap;
|
|
|
import java.util.TreeMap;
|
|
|
+import java.util.concurrent.locks.ReentrantLock;
|
|
|
|
|
|
import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -84,7 +85,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
/**
|
|
|
* The Cache Manager handles caching on DataNodes.
|
|
|
*
|
|
|
- * This class is instantiated by the FSNamesystem when caching is enabled.
|
|
|
+ * This class is instantiated by the FSNamesystem.
|
|
|
* It maintains the mapping of cached blocks to datanodes via processing
|
|
|
* datanode cache reports. Based on these reports and addition and removal of
|
|
|
* caching directives, we will schedule caching and uncaching work.
|
|
@@ -93,6 +94,8 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
public final class CacheManager {
|
|
|
public static final Log LOG = LogFactory.getLog(CacheManager.class);
|
|
|
|
|
|
+ private static final float MIN_CACHED_BLOCKS_PERCENT = 0.001f;
|
|
|
+
|
|
|
// TODO: add pending / underCached / schedule cached blocks stats.
|
|
|
|
|
|
/**
|
|
@@ -148,32 +151,14 @@ public final class CacheManager {
|
|
|
private final long scanIntervalMs;
|
|
|
|
|
|
/**
|
|
|
- * Whether caching is enabled.
|
|
|
- *
|
|
|
- * If caching is disabled, we will not process cache reports or store
|
|
|
- * information about what is cached where. We also do not start the
|
|
|
- * CacheReplicationMonitor thread. This will save resources, but provide
|
|
|
- * less functionality.
|
|
|
- *
|
|
|
- * Even when caching is disabled, we still store path-based cache
|
|
|
- * information. This information is stored in the edit log and fsimage. We
|
|
|
- * don't want to lose it just because a configuration setting was turned off.
|
|
|
- * However, we will not act on this information if caching is disabled.
|
|
|
+ * All cached blocks.
|
|
|
*/
|
|
|
- private final boolean enabled;
|
|
|
+ private final GSet<CachedBlock, CachedBlock> cachedBlocks;
|
|
|
|
|
|
/**
|
|
|
- * Whether the CacheManager is active.
|
|
|
- *
|
|
|
- * When the CacheManager is active, it tells the DataNodes what to cache
|
|
|
- * and uncache. The CacheManager cannot become active if enabled = false.
|
|
|
+ * Lock which protects the CacheReplicationMonitor.
|
|
|
*/
|
|
|
- private boolean active = false;
|
|
|
-
|
|
|
- /**
|
|
|
- * All cached blocks.
|
|
|
- */
|
|
|
- private final GSet<CachedBlock, CachedBlock> cachedBlocks;
|
|
|
+ private final ReentrantLock crmLock = new ReentrantLock();
|
|
|
|
|
|
/**
|
|
|
* The CacheReplicationMonitor.
|
|
@@ -194,54 +179,51 @@ public final class CacheManager {
|
|
|
scanIntervalMs = conf.getLong(
|
|
|
DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS,
|
|
|
DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT);
|
|
|
- this.enabled = conf.getBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY,
|
|
|
- DFS_NAMENODE_CACHING_ENABLED_DEFAULT);
|
|
|
- this.cachedBlocks = !enabled ? null :
|
|
|
- new LightWeightGSet<CachedBlock, CachedBlock>(
|
|
|
- LightWeightGSet.computeCapacity(0.25, "cachedBlocks"));
|
|
|
+ float cachedBlocksPercent = conf.getFloat(
|
|
|
+ DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT,
|
|
|
+ DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT_DEFAULT);
|
|
|
+ if (cachedBlocksPercent < MIN_CACHED_BLOCKS_PERCENT) {
|
|
|
+ LOG.info("Using minimum value " + MIN_CACHED_BLOCKS_PERCENT +
|
|
|
+ " for " + DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT);
|
|
|
+ cachedBlocksPercent = MIN_CACHED_BLOCKS_PERCENT;
|
|
|
+ }
|
|
|
+ this.cachedBlocks = new LightWeightGSet<CachedBlock, CachedBlock>(
|
|
|
+ LightWeightGSet.computeCapacity(cachedBlocksPercent,
|
|
|
+ "cachedBlocks"));
|
|
|
+
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Activate the cache manager.
|
|
|
- *
|
|
|
- * When the cache manager is active, tell the datanodes where to cache files.
|
|
|
- */
|
|
|
- public void activate() {
|
|
|
- assert namesystem.hasWriteLock();
|
|
|
- if (enabled && (!active)) {
|
|
|
- LOG.info("Activating CacheManager. " +
|
|
|
- "Starting replication monitor thread...");
|
|
|
- active = true;
|
|
|
- monitor = new CacheReplicationMonitor(namesystem, this,
|
|
|
- scanIntervalMs);
|
|
|
- monitor.start();
|
|
|
+ public void startMonitorThread() {
|
|
|
+ crmLock.lock();
|
|
|
+ try {
|
|
|
+ if (this.monitor == null) {
|
|
|
+ this.monitor = new CacheReplicationMonitor(namesystem, this,
|
|
|
+ scanIntervalMs, crmLock);
|
|
|
+ this.monitor.start();
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ crmLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Deactivate the cache manager.
|
|
|
- *
|
|
|
- * When the cache manager is inactive, it does not tell the datanodes where to
|
|
|
- * cache files.
|
|
|
- */
|
|
|
- public void deactivate() {
|
|
|
- assert namesystem.hasWriteLock();
|
|
|
- if (active) {
|
|
|
- LOG.info("Deactivating CacheManager. " +
|
|
|
- "stopping CacheReplicationMonitor thread...");
|
|
|
- active = false;
|
|
|
- IOUtils.closeQuietly(monitor);
|
|
|
- monitor = null;
|
|
|
- LOG.info("CacheReplicationMonitor thread stopped and deactivated.");
|
|
|
+ public void stopMonitorThread() {
|
|
|
+ crmLock.lock();
|
|
|
+ try {
|
|
|
+ if (this.monitor != null) {
|
|
|
+ CacheReplicationMonitor prevMonitor = this.monitor;
|
|
|
+ this.monitor = null;
|
|
|
+ IOUtils.closeQuietly(prevMonitor);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ crmLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Return true only if the cache manager is active.
|
|
|
- * Must be called under the FSN read or write lock.
|
|
|
- */
|
|
|
- public boolean isActive() {
|
|
|
- return active;
|
|
|
+ public void clearDirectiveStats() {
|
|
|
+ assert namesystem.hasWriteLock();
|
|
|
+ for (CacheDirective directive : directivesById.values()) {
|
|
|
+ directive.resetStatistics();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -480,9 +462,7 @@ public final class CacheManager {
|
|
|
directive.addBytesNeeded(stats.getBytesNeeded());
|
|
|
directive.addFilesNeeded(directive.getFilesNeeded());
|
|
|
|
|
|
- if (monitor != null) {
|
|
|
- monitor.setNeedsRescan();
|
|
|
- }
|
|
|
+ setNeedsRescan();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -514,10 +494,6 @@ public final class CacheManager {
|
|
|
long expiryTime = validateExpiryTime(info, pool.getMaxRelativeExpiryMs());
|
|
|
// Do quota validation if required
|
|
|
if (!flags.contains(CacheFlag.FORCE)) {
|
|
|
- // Can't kick and wait if caching is disabled
|
|
|
- if (monitor != null) {
|
|
|
- monitor.waitForRescan();
|
|
|
- }
|
|
|
checkLimit(pool, path, replication);
|
|
|
}
|
|
|
// All validation passed
|
|
@@ -622,9 +598,7 @@ public final class CacheManager {
|
|
|
validateExpiryTime(infoWithDefaults, destPool.getMaxRelativeExpiryMs());
|
|
|
|
|
|
// Indicate changes to the CRM
|
|
|
- if (monitor != null) {
|
|
|
- monitor.setNeedsRescan();
|
|
|
- }
|
|
|
+ setNeedsRescan();
|
|
|
|
|
|
// Validation passed
|
|
|
removeInternal(prevEntry);
|
|
@@ -659,9 +633,7 @@ public final class CacheManager {
|
|
|
pool.getDirectiveList().remove(directive);
|
|
|
assert directive.getPool() == null;
|
|
|
|
|
|
- if (monitor != null) {
|
|
|
- monitor.setNeedsRescan();
|
|
|
- }
|
|
|
+ setNeedsRescan();
|
|
|
}
|
|
|
|
|
|
public void removeDirective(long id, FSPermissionChecker pc)
|
|
@@ -694,9 +666,6 @@ public final class CacheManager {
|
|
|
if (filter.getReplication() != null) {
|
|
|
throw new IOException("Filtering by replication is unsupported.");
|
|
|
}
|
|
|
- if (monitor != null) {
|
|
|
- monitor.waitForRescanIfNeeded();
|
|
|
- }
|
|
|
ArrayList<CacheDirectiveEntry> replies =
|
|
|
new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
|
|
|
int numReplies = 0;
|
|
@@ -805,9 +774,7 @@ public final class CacheManager {
|
|
|
bld.append(prefix).append("set limit to " + info.getLimit());
|
|
|
prefix = "; ";
|
|
|
// New limit changes stats, need to set needs refresh
|
|
|
- if (monitor != null) {
|
|
|
- monitor.setNeedsRescan();
|
|
|
- }
|
|
|
+ setNeedsRescan();
|
|
|
}
|
|
|
if (info.getMaxRelativeExpiryMs() != null) {
|
|
|
final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
|
|
@@ -853,9 +820,7 @@ public final class CacheManager {
|
|
|
directivesById.remove(directive.getId());
|
|
|
iter.remove();
|
|
|
}
|
|
|
- if (monitor != null) {
|
|
|
- monitor.setNeedsRescan();
|
|
|
- }
|
|
|
+ setNeedsRescan();
|
|
|
} catch (IOException e) {
|
|
|
LOG.info("removeCachePool of " + poolName + " failed: ", e);
|
|
|
throw e;
|
|
@@ -866,9 +831,6 @@ public final class CacheManager {
|
|
|
public BatchedListEntries<CachePoolEntry>
|
|
|
listCachePools(FSPermissionChecker pc, String prevKey) {
|
|
|
assert namesystem.hasReadLock();
|
|
|
- if (monitor != null) {
|
|
|
- monitor.waitForRescanIfNeeded();
|
|
|
- }
|
|
|
final int NUM_PRE_ALLOCATED_ENTRIES = 16;
|
|
|
ArrayList<CachePoolEntry> results =
|
|
|
new ArrayList<CachePoolEntry>(NUM_PRE_ALLOCATED_ENTRIES);
|
|
@@ -884,9 +846,6 @@ public final class CacheManager {
|
|
|
}
|
|
|
|
|
|
public void setCachedLocations(LocatedBlock block) {
|
|
|
- if (!enabled) {
|
|
|
- return;
|
|
|
- }
|
|
|
CachedBlock cachedBlock =
|
|
|
new CachedBlock(block.getBlock().getBlockId(),
|
|
|
(short)0, false);
|
|
@@ -902,12 +861,6 @@ public final class CacheManager {
|
|
|
|
|
|
public final void processCacheReport(final DatanodeID datanodeID,
|
|
|
final List<Long> blockIds) throws IOException {
|
|
|
- if (!enabled) {
|
|
|
- LOG.info("Ignoring cache report from " + datanodeID +
|
|
|
- " because " + DFS_NAMENODE_CACHING_ENABLED_KEY + " = false. " +
|
|
|
- "number of blocks: " + blockIds.size());
|
|
|
- return;
|
|
|
- }
|
|
|
namesystem.writeLock();
|
|
|
final long startTime = Time.monotonicNow();
|
|
|
final long endTime;
|
|
@@ -1085,4 +1038,36 @@ public final class CacheManager {
|
|
|
}
|
|
|
prog.endStep(Phase.LOADING_FSIMAGE, step);
|
|
|
}
|
|
|
+
|
|
|
+ public void waitForRescanIfNeeded() {
|
|
|
+ crmLock.lock();
|
|
|
+ try {
|
|
|
+ if (monitor != null) {
|
|
|
+ monitor.waitForRescanIfNeeded();
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ crmLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setNeedsRescan() {
|
|
|
+ crmLock.lock();
|
|
|
+ try {
|
|
|
+ if (monitor != null) {
|
|
|
+ monitor.setNeedsRescan();
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ crmLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public Thread getCacheReplicationMonitor() {
|
|
|
+ crmLock.lock();
|
|
|
+ try {
|
|
|
+ return monitor;
|
|
|
+ } finally {
|
|
|
+ crmLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|