HDFS-6613. Improve logging in caching classes. (wang)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1607697 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Wang 2014-07-03 17:13:59 +00:00
parent 160c912ee6
commit 93e23a9915
5 changed files with 133 additions and 176 deletions

View File

@ -262,6 +262,8 @@ Release 2.6.0 - UNRELEASED
IMPROVEMENTS
HDFS-6613. Improve logging in caching classes. (wang)
OPTIMIZATIONS
BUG FIXES

View File

@ -33,8 +33,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.protocol.Block;
@ -53,8 +51,11 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.util.ReadOnlyList;
import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Preconditions;
;
/**
* Scans the namesystem, scheduling blocks to be cached as appropriate.
@ -65,8 +66,8 @@ import com.google.common.base.Preconditions;
@InterfaceAudience.LimitedPrivate({"HDFS"})
public class CacheReplicationMonitor extends Thread implements Closeable {
private static final Log LOG =
LogFactory.getLog(CacheReplicationMonitor.class);
private static final Logger LOG =
LoggerFactory.getLogger(CacheReplicationMonitor.class);
private final FSNamesystem namesystem;
@ -207,7 +208,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
LOG.info("Shutting down CacheReplicationMonitor.");
return;
} catch (Throwable t) {
LOG.fatal("Thread exiting", t);
LOG.error("Thread exiting", t);
terminate(1, t);
}
}
@ -316,11 +317,8 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
scannedDirectives++;
// Skip processing this entry if it has expired
if (directive.getExpiryTime() > 0 && directive.getExpiryTime() <= now) {
if (LOG.isDebugEnabled()) {
LOG.debug("Directive " + directive.getId() + ": the directive " +
"expired at " + directive.getExpiryTime() + " (now = " +
now + ")");
}
LOG.debug("Directive {}: the directive expired at {} (now = {})",
directive.getId(), directive.getExpiryTime(), now);
continue;
}
String path = directive.getPath();
@ -329,17 +327,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
node = fsDir.getINode(path);
} catch (UnresolvedLinkException e) {
// We don't cache through symlinks
if (LOG.isDebugEnabled()) {
LOG.debug("Directive " + directive.getId() +
": got UnresolvedLinkException while resolving path " + path);
}
LOG.debug("Directive {}: got UnresolvedLinkException while resolving "
+ "path {}", directive.getId(), path
);
continue;
}
if (node == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Directive " + directive.getId() +
": No inode found at " + path);
}
LOG.debug("Directive {}: No inode found at {}", directive.getId(),
path);
} else if (node.isDirectory()) {
INodeDirectory dir = node.asDirectory();
ReadOnlyList<INode> children = dir
@ -352,10 +347,8 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
} else if (node.isFile()) {
rescanFile(directive, node.asFile());
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Directive " + directive.getId() +
": ignoring non-directive, non-file inode " + node);
}
LOG.debug("Directive {}: ignoring non-directive, non-file inode {} ",
directive.getId(), node);
}
}
}
@ -381,15 +374,13 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
// do not cache this file.
CachePool pool = directive.getPool();
if (pool.getBytesNeeded() > pool.getLimit()) {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("Directive %d: not scanning file %s because " +
"bytesNeeded for pool %s is %d, but the pool's limit is %d",
LOG.debug("Directive {}: not scanning file {} because " +
"bytesNeeded for pool {} is {}, but the pool's limit is {}",
directive.getId(),
file.getFullPathName(),
pool.getPoolName(),
pool.getBytesNeeded(),
pool.getLimit()));
}
pool.getLimit());
return;
}
@ -397,11 +388,10 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
for (BlockInfo blockInfo : blockInfos) {
if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
// We don't try to cache blocks that are under construction.
if (LOG.isTraceEnabled()) {
LOG.trace("Directive " + directive.getId() + ": can't cache " +
"block " + blockInfo + " because it is in state " +
blockInfo.getBlockUCState() + ", not COMPLETE.");
}
LOG.trace("Directive {}: can't cache block {} because it is in state "
+ "{}, not COMPLETE.", directive.getId(), blockInfo,
blockInfo.getBlockUCState()
);
continue;
}
Block block = new Block(blockInfo.getBlockId());
@ -415,7 +405,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
// Update bytesUsed using the current replication levels.
// Assumptions: we assume that all the blocks are the same length
// on each datanode. We can assume this because we're only caching
// blocks in state COMMITTED.
// blocks in state COMPLETE.
// Note that if two directives are caching the same block(s), they will
// both get them added to their bytesCached.
List<DatanodeDescriptor> cachedOn =
@ -441,21 +431,16 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
ocblock.setReplicationAndMark(directive.getReplication(), mark);
}
}
if (LOG.isTraceEnabled()) {
LOG.trace("Directive " + directive.getId() + ": setting replication " +
"for block " + blockInfo + " to " + ocblock.getReplication());
}
LOG.trace("Directive {}: setting replication for block {} to {}",
directive.getId(), blockInfo, ocblock.getReplication());
}
// Increment the "cached" statistics
directive.addBytesCached(cachedTotal);
if (cachedTotal == neededTotal) {
directive.addFilesCached(1);
}
if (LOG.isDebugEnabled()) {
LOG.debug("Directive " + directive.getId() + ": caching " +
file.getFullPathName() + ": " + cachedTotal + "/" + neededTotal +
" bytes");
}
LOG.debug("Directive {}: caching {}: {}/{} bytes", directive.getId(),
file.getFullPathName(), cachedTotal, neededTotal);
}
private String findReasonForNotCaching(CachedBlock cblock,
@ -512,11 +497,9 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
iter.hasNext(); ) {
DatanodeDescriptor datanode = iter.next();
if (!cblock.isInList(datanode.getCached())) {
if (LOG.isTraceEnabled()) {
LOG.trace("Block " + cblock.getBlockId() + ": removing from " +
"PENDING_UNCACHED for node " + datanode.getDatanodeUuid() +
"because the DataNode uncached it.");
}
LOG.trace("Block {}: removing from PENDING_UNCACHED for node {} "
+ "because the DataNode uncached it.", cblock.getBlockId(),
datanode.getDatanodeUuid());
datanode.getPendingUncached().remove(cblock);
iter.remove();
}
@ -526,10 +509,8 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
String reason = findReasonForNotCaching(cblock, blockInfo);
int neededCached = 0;
if (reason != null) {
if (LOG.isTraceEnabled()) {
LOG.trace("Block " + cblock.getBlockId() + ": can't cache " +
"block because it is " + reason);
}
LOG.trace("Block {}: can't cache block because it is {}",
cblock.getBlockId(), reason);
} else {
neededCached = cblock.getReplication();
}
@ -541,12 +522,12 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
DatanodeDescriptor datanode = iter.next();
datanode.getPendingCached().remove(cblock);
iter.remove();
if (LOG.isTraceEnabled()) {
LOG.trace("Block " + cblock.getBlockId() + ": removing from " +
"PENDING_CACHED for node " + datanode.getDatanodeUuid() +
"because we already have " + numCached + " cached " +
"replicas and we only need " + neededCached);
}
LOG.trace("Block {}: removing from PENDING_CACHED for node {}"
+ "because we already have {} cached replicas and we only" +
" need {}",
cblock.getBlockId(), datanode.getDatanodeUuid(), numCached,
neededCached
);
}
}
if (numCached < neededCached) {
@ -556,12 +537,11 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
DatanodeDescriptor datanode = iter.next();
datanode.getPendingUncached().remove(cblock);
iter.remove();
if (LOG.isTraceEnabled()) {
LOG.trace("Block " + cblock.getBlockId() + ": removing from " +
"PENDING_UNCACHED for node " + datanode.getDatanodeUuid() +
"because we only have " + numCached + " cached replicas " +
"and we need " + neededCached);
}
LOG.trace("Block {}: removing from PENDING_UNCACHED for node {} "
+ "because we only have {} cached replicas and we need " +
"{}", cblock.getBlockId(), datanode.getDatanodeUuid(),
numCached, neededCached
);
}
}
int neededUncached = numCached -
@ -581,11 +561,10 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
pendingUncached.isEmpty() &&
pendingCached.isEmpty()) {
// we have nothing more to do with this block.
if (LOG.isTraceEnabled()) {
LOG.trace("Block " + cblock.getBlockId() + ": removing from " +
"cachedBlocks, since neededCached == 0, and " +
"pendingUncached and pendingCached are empty.");
}
LOG.trace("Block {}: removing from cachedBlocks, since neededCached "
+ "== 0, and pendingUncached and pendingCached are empty.",
cblock.getBlockId()
);
cbIter.remove();
}
}
@ -643,18 +622,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
BlockInfo blockInfo = blockManager.
getStoredBlock(new Block(cachedBlock.getBlockId()));
if (blockInfo == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Block " + cachedBlock.getBlockId() + ": can't add new " +
"cached replicas, because there is no record of this block " +
"on the NameNode.");
}
LOG.debug("Block {}: can't add new cached replicas," +
" because there is no record of this block " +
"on the NameNode.", cachedBlock.getBlockId());
return;
}
if (!blockInfo.isComplete()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Block " + cachedBlock.getBlockId() + ": can't cache this " +
"block, because it is not yet complete.");
}
LOG.debug("Block {}: can't cache this block, because it is not yet"
+ " complete.", cachedBlock.getBlockId());
return;
}
// Filter the list of replicas to only the valid targets
@ -678,7 +653,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
if (pendingCached.contains(datanode) || cached.contains(datanode)) {
continue;
}
long pendingCapacity = datanode.getCacheRemaining();
long pendingBytes = 0;
// Subtract pending cached blocks from effective capacity
Iterator<CachedBlock> it = datanode.getPendingCached().iterator();
while (it.hasNext()) {
@ -686,7 +661,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
BlockInfo info =
blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
if (info != null) {
pendingCapacity -= info.getNumBytes();
pendingBytes -= info.getNumBytes();
}
}
it = datanode.getPendingUncached().iterator();
@ -696,17 +671,17 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
BlockInfo info =
blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
if (info != null) {
pendingCapacity += info.getNumBytes();
pendingBytes += info.getNumBytes();
}
}
long pendingCapacity = pendingBytes + datanode.getCacheRemaining();
if (pendingCapacity < blockInfo.getNumBytes()) {
if (LOG.isTraceEnabled()) {
LOG.trace("Block " + blockInfo.getBlockId() + ": DataNode " +
datanode.getDatanodeUuid() + " is not a valid possibility " +
"because the block has size " + blockInfo.getNumBytes() + ", but " +
"the DataNode only has " + datanode.getCacheRemaining() + " " +
"bytes of cache remaining.");
}
LOG.trace("Block {}: DataNode {} is not a valid possibility " +
"because the block has size {}, but the DataNode only has {}" +
"bytes of cache remaining ({} pending bytes, {} already cached.",
blockInfo.getBlockId(), datanode.getDatanodeUuid(),
blockInfo.getNumBytes(), pendingCapacity, pendingBytes,
datanode.getCacheRemaining());
outOfCapacity++;
continue;
}
@ -715,22 +690,20 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
List<DatanodeDescriptor> chosen = chooseDatanodesForCaching(possibilities,
neededCached, blockManager.getDatanodeManager().getStaleInterval());
for (DatanodeDescriptor datanode : chosen) {
if (LOG.isTraceEnabled()) {
LOG.trace("Block " + blockInfo.getBlockId() + ": added to " +
"PENDING_CACHED on DataNode " + datanode.getDatanodeUuid());
}
LOG.trace("Block {}: added to PENDING_CACHED on DataNode {}",
blockInfo.getBlockId(), datanode.getDatanodeUuid());
pendingCached.add(datanode);
boolean added = datanode.getPendingCached().add(cachedBlock);
assert added;
}
// We were unable to satisfy the requested replication factor
if (neededCached > chosen.size()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Block " + blockInfo.getBlockId() + ": we only have " +
(cachedBlock.getReplication() - neededCached + chosen.size()) +
" of " + cachedBlock.getReplication() + " cached replicas. " +
outOfCapacity + " DataNodes have insufficient cache capacity.");
}
LOG.debug("Block {}: we only have {} of {} cached replicas."
+ " {} DataNodes have insufficient cache capacity.",
blockInfo.getBlockId(),
(cachedBlock.getReplication() - neededCached + chosen.size()),
cachedBlock.getReplication(), outOfCapacity
);
}
}

View File

@ -37,8 +37,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.ChecksumException;
@ -47,6 +45,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Manages caching for an FsDatasetImpl by using the mmap(2) and mlock(2)
@ -101,7 +101,8 @@ public class FsDatasetCache {
}
}
private static final Log LOG = LogFactory.getLog(FsDatasetCache.class);
private static final Logger LOG = LoggerFactory.getLogger(FsDatasetCache
.class);
/**
* Stores MappableBlock objects and the states they're in.
@ -245,21 +246,17 @@ public class FsDatasetCache {
ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
Value prevValue = mappableBlockMap.get(key);
if (prevValue != null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Block with id " + blockId + ", pool " + bpid +
" already exists in the FsDatasetCache with state " +
prevValue.state);
}
LOG.debug("Block with id {}, pool {} already exists in the "
+ "FsDatasetCache with state {}", blockId, bpid, prevValue.state
);
numBlocksFailedToCache.incrementAndGet();
return;
}
mappableBlockMap.put(key, new Value(null, State.CACHING));
volumeExecutor.execute(
new CachingTask(key, blockFileName, length, genstamp));
if (LOG.isDebugEnabled()) {
LOG.debug("Initiating caching for Block with id " + blockId +
", pool " + bpid);
}
LOG.debug("Initiating caching for Block with id {}, pool {}", blockId,
bpid);
}
synchronized void uncacheBlock(String bpid, long blockId) {
@ -270,44 +267,34 @@ public class FsDatasetCache {
processBlockMunlockRequest(key)) {
// TODO: we probably want to forcibly uncache the block (and close the
// shm) after a certain timeout has elapsed.
if (LOG.isDebugEnabled()) {
LOG.debug(key + " is anchored, and can't be uncached now.");
}
LOG.debug("{} is anchored, and can't be uncached now.", key);
return;
}
if (prevValue == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
"does not need to be uncached, because it is not currently " +
"in the mappableBlockMap.");
}
LOG.debug("Block with id {}, pool {} does not need to be uncached, "
+ "because it is not currently in the mappableBlockMap.", blockId,
bpid);
numBlocksFailedToUncache.incrementAndGet();
return;
}
switch (prevValue.state) {
case CACHING:
if (LOG.isDebugEnabled()) {
LOG.debug("Cancelling caching for block with id " + blockId +
", pool " + bpid + ".");
}
LOG.debug("Cancelling caching for block with id {}, pool {}.", blockId,
bpid);
mappableBlockMap.put(key,
new Value(prevValue.mappableBlock, State.CACHING_CANCELLED));
break;
case CACHED:
if (LOG.isDebugEnabled()) {
LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
"has been scheduled for uncaching.");
}
LOG.debug(
"Block with id {}, pool {} has been scheduled for uncaching" + ".",
blockId, bpid);
mappableBlockMap.put(key,
new Value(prevValue.mappableBlock, State.UNCACHING));
uncachingExecutor.execute(new UncachingTask(key));
break;
default:
if (LOG.isDebugEnabled()) {
LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
"does not need to be uncached, because it is " +
"in state " + prevValue.state + ".");
}
LOG.debug("Block with id {}, pool {} does not need to be uncached, "
+ "because it is in state {}.", blockId, bpid, prevValue.state);
numBlocksFailedToUncache.incrementAndGet();
break;
}
@ -386,10 +373,8 @@ public class FsDatasetCache {
}
mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED));
}
if (LOG.isDebugEnabled()) {
LOG.debug("Successfully cached " + key + ". We are now caching " +
newUsedBytes + " bytes in total.");
}
LOG.debug("Successfully cached {}. We are now caching {} bytes in"
+ " total.", key, newUsedBytes);
dataset.datanode.getShortCircuitRegistry().processBlockMlockEvent(key);
numBlocksCached.addAndGet(1);
dataset.datanode.getMetrics().incrBlocksCached(1);
@ -399,12 +384,10 @@ public class FsDatasetCache {
IOUtils.closeQuietly(metaIn);
if (!success) {
if (reservedBytes) {
newUsedBytes = usedBytesCount.release(length);
}
if (LOG.isDebugEnabled()) {
LOG.debug("Caching of " + key + " was aborted. We are now " +
"caching only " + newUsedBytes + " + bytes in total.");
usedBytesCount.release(length);
}
LOG.debug("Caching of {} was aborted. We are now caching only {} "
+ "bytes in total.", key, usedBytesCount.get());
if (mappableBlock != null) {
mappableBlock.close();
}
@ -444,10 +427,7 @@ public class FsDatasetCache {
usedBytesCount.release(value.mappableBlock.getLength());
numBlocksCached.addAndGet(-1);
dataset.datanode.getMetrics().incrBlocksUncached(1);
if (LOG.isDebugEnabled()) {
LOG.debug("Uncaching of " + key + " completed. " +
"usedBytes = " + newUsedBytes);
}
LOG.debug("Uncaching of {} completed. usedBytes = {}", key, newUsedBytes);
}
}

View File

@ -43,8 +43,6 @@ import java.util.TreeMap;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
@ -85,6 +83,8 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.LightWeightGSet;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
@ -99,7 +99,7 @@ import com.google.common.collect.Lists;
*/
@InterfaceAudience.LimitedPrivate({"HDFS"})
public final class CacheManager {
public static final Log LOG = LogFactory.getLog(CacheManager.class);
public static final Logger LOG = LoggerFactory.getLogger(CacheManager.class);
private static final float MIN_CACHED_BLOCKS_PERCENT = 0.001f;
@ -205,8 +205,8 @@ public final class CacheManager {
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);
LOG.info("Using minimum value {} for {}", MIN_CACHED_BLOCKS_PERCENT,
DFS_NAMENODE_PATH_BASED_CACHE_BLOCK_MAP_ALLOCATION_PERCENT);
cachedBlocksPercent = MIN_CACHED_BLOCKS_PERCENT;
}
this.cachedBlocks = new LightWeightGSet<CachedBlock, CachedBlock>(
@ -346,10 +346,8 @@ public final class CacheManager {
*/
private static long validateExpiryTime(CacheDirectiveInfo info,
long maxRelativeExpiryTime) throws InvalidRequestException {
if (LOG.isTraceEnabled()) {
LOG.trace("Validating directive " + info
+ " pool maxRelativeExpiryTime " + maxRelativeExpiryTime);
}
LOG.trace("Validating directive {} pool maxRelativeExpiryTime {}", info,
maxRelativeExpiryTime);
final long now = new Date().getTime();
final long maxAbsoluteExpiryTime = now + maxRelativeExpiryTime;
if (info == null || info.getExpiration() == null) {
@ -539,7 +537,7 @@ public final class CacheManager {
LOG.warn("addDirective of " + info + " failed: ", e);
throw e;
}
LOG.info("addDirective of " + info + " successful.");
LOG.info("addDirective of {} successful.", info);
return directive.toInfo();
}
@ -641,8 +639,7 @@ public final class CacheManager {
LOG.warn("modifyDirective of " + idString + " failed: ", e);
throw e;
}
LOG.info("modifyDirective of " + idString + " successfully applied " +
info+ ".");
LOG.info("modifyDirective of {} successfully applied {}.", idString, info);
}
private void removeInternal(CacheDirective directive)
@ -779,7 +776,7 @@ public final class CacheManager {
LOG.info("addCachePool of " + info + " failed: ", e);
throw e;
}
LOG.info("addCachePool of " + info + " successful.");
LOG.info("addCachePool of {} successful.", info);
return pool.getInfo(true);
}
@ -842,8 +839,8 @@ public final class CacheManager {
LOG.info("modifyCachePool of " + info + " failed: ", e);
throw e;
}
LOG.info("modifyCachePool of " + info.getPoolName() + " successful; "
+ bld.toString());
LOG.info("modifyCachePool of {} successful; {}", info.getPoolName(),
bld.toString());
}
/**
@ -935,11 +932,9 @@ public final class CacheManager {
if (metrics != null) {
metrics.addCacheBlockReport((int) (endTime - startTime));
}
if (LOG.isDebugEnabled()) {
LOG.debug("Processed cache report from "
+ datanodeID + ", blocks: " + blockIds.size()
+ ", processing time: " + (endTime - startTime) + " msecs");
}
LOG.debug("Processed cache report from {}, blocks: {}, " +
"processing time: {} msecs", datanodeID, blockIds.size(),
(endTime - startTime));
}
private void processCacheReportImpl(final DatanodeDescriptor datanode,
@ -950,6 +945,8 @@ public final class CacheManager {
CachedBlocksList pendingCachedList = datanode.getPendingCached();
for (Iterator<Long> iter = blockIds.iterator(); iter.hasNext(); ) {
long blockId = iter.next();
LOG.trace("Cache report from datanode {} has block {}", datanode,
blockId);
CachedBlock cachedBlock =
new CachedBlock(blockId, (short)0, false);
CachedBlock prevCachedBlock = cachedBlocks.get(cachedBlock);
@ -959,15 +956,18 @@ public final class CacheManager {
cachedBlock = prevCachedBlock;
} else {
cachedBlocks.put(cachedBlock);
LOG.trace("Added block {} to cachedBlocks", cachedBlock);
}
// Add the block to the datanode's implicit cached block list
// if it's not already there. Similarly, remove it from the pending
// cached block list if it exists there.
if (!cachedBlock.isPresent(cachedList)) {
cachedList.add(cachedBlock);
LOG.trace("Added block {} to CACHED list.", cachedBlock);
}
if (cachedBlock.isPresent(pendingCachedList)) {
pendingCachedList.remove(cachedBlock);
LOG.trace("Removed block {} from PENDING_CACHED list.", cachedBlock);
}
}
}

View File

@ -34,6 +34,7 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.EnumSet;
import java.util.Iterator;
@ -682,6 +683,12 @@ public class TestCacheDirectives {
} finally {
namesystem.readUnlock();
}
LOG.info(logString + " cached blocks: have " + numCachedBlocks +
" / " + expectedCachedBlocks + ". " +
"cached replicas: have " + numCachedReplicas +
" / " + expectedCachedReplicas);
if (expectedCachedBlocks == -1 ||
numCachedBlocks == expectedCachedBlocks) {
if (expectedCachedReplicas == -1 ||
@ -689,10 +696,6 @@ public class TestCacheDirectives {
return true;
}
}
LOG.info(logString + " cached blocks: have " + numCachedBlocks +
" / " + expectedCachedBlocks + ". " +
"cached replicas: have " + numCachedReplicas +
" / " + expectedCachedReplicas);
return false;
}
}, 500, 60000);
@ -1415,7 +1418,10 @@ public class TestCacheDirectives {
for (DataNode dn : cluster.getDataNodes()) {
DatanodeDescriptor descriptor =
datanodeManager.getDatanode(dn.getDatanodeId());
Assert.assertTrue(descriptor.getPendingCached().isEmpty());
Assert.assertTrue("Pending cached list of " + descriptor +
" is not empty, "
+ Arrays.toString(descriptor.getPendingCached().toArray()),
descriptor.getPendingCached().isEmpty());
}
} finally {
cluster.getNamesystem().readUnlock();
@ -1430,10 +1436,6 @@ public class TestCacheDirectives {
int numCachedReplicas = (int) ((CACHE_CAPACITY*NUM_DATANODES)/BLOCK_SIZE);
DFSTestUtil.createFile(dfs, fileName, fileLen, (short) NUM_DATANODES,
0xFADED);
// Set up a log appender watcher
final LogVerificationAppender appender = new LogVerificationAppender();
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
dfs.addCachePool(new CachePoolInfo("pool"));
dfs.addCacheDirective(new CacheDirectiveInfo.Builder().setPool("pool")
.setPath(fileName).setReplication((short) 1).build());