HDFS-16648. Add isDebugEnabled check for debug blockLogs in some classes (#4529)

This commit is contained in:
xuzq 2022-08-06 21:34:01 +08:00 committed by GitHub
parent bd0f9a46e1
commit 25ccdc77af
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 218 additions and 315 deletions

View File

@ -287,11 +287,8 @@ public abstract class Server {
throw new IllegalArgumentException("ReRegistration of rpcKind: " +
rpcKind);
}
if (LOG.isDebugEnabled()) {
LOG.debug("rpcKind=" + rpcKind +
", rpcRequestWrapperClass=" + rpcRequestWrapperClass +
", rpcInvoker=" + rpcInvoker);
}
LOG.debug("rpcKind={}, rpcRequestWrapperClass={}, rpcInvoker={}.",
rpcKind, rpcRequestWrapperClass, rpcInvoker);
}
public Class<? extends Writable> getRpcRequestWrapper(
@ -1212,9 +1209,7 @@ public abstract class Server {
deltaNanos = Time.monotonicNowNanos() - startNanos;
details.set(Timing.RESPONSE, deltaNanos, TimeUnit.NANOSECONDS);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Deferring response for callId: " + this.callId);
}
LOG.debug("Deferring response for callId: {}", this.callId);
}
return null;
}
@ -1711,9 +1706,7 @@ public abstract class Server {
// If there were some calls that have not been sent out for a
// long time, discard them.
//
if(LOG.isDebugEnabled()) {
LOG.debug("Checking for old call responses.");
}
LOG.debug("Checking for old call responses.");
ArrayList<RpcCall> calls;
// get the list of channels from list of keys.
@ -1813,9 +1806,8 @@ public abstract class Server {
//
call = responseQueue.removeFirst();
SocketChannel channel = call.connection.channel;
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() + ": responding to " + call);
}
LOG.debug("{}: responding to {}.", Thread.currentThread().getName(), call);
//
// Send as much data as we can in the non-blocking fashion
//
@ -1832,10 +1824,8 @@ public abstract class Server {
} else {
done = false; // more calls pending to be sent.
}
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() + ": responding to " + call
+ " Wrote " + numBytes + " bytes.");
}
LOG.debug("{}: responding to {} Wrote {} bytes.",
Thread.currentThread().getName(), call, numBytes);
} else {
//
// If we were unable to write the entire response out, then
@ -1860,10 +1850,8 @@ public abstract class Server {
decPending();
}
}
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() + ": responding to " + call
+ " Wrote partial " + numBytes + " bytes.");
}
LOG.debug("{}: responding to {} Wrote partial {} bytes.",
Thread.currentThread().getName(), call, numBytes);
}
error = false; // everything went off well
}
@ -2209,13 +2197,11 @@ public abstract class Server {
if (saslServer != null && saslServer.isComplete()) {
if (LOG.isDebugEnabled()) {
LOG.debug("SASL server context established. Negotiated QoP is "
+ saslServer.getNegotiatedProperty(Sasl.QOP));
LOG.debug("SASL server context established. Negotiated QoP is {}.",
saslServer.getNegotiatedProperty(Sasl.QOP));
}
user = getAuthorizedUgi(saslServer.getAuthorizationID());
if (LOG.isDebugEnabled()) {
LOG.debug("SASL server successfully authenticated client: " + user);
}
LOG.debug("SASL server successfully authenticated client: {}.", user);
rpcMetrics.incrAuthenticationSuccesses();
AUDITLOG.info(AUTH_SUCCESSFUL_FOR + user + " from " + toString());
saslContextEstablished = true;
@ -2320,10 +2306,8 @@ public abstract class Server {
throw new SaslException("Client did not send a token");
}
byte[] saslToken = saslMessage.getToken().toByteArray();
if (LOG.isDebugEnabled()) {
LOG.debug("Have read input token of size " + saslToken.length
+ " for processing by saslServer.evaluateResponse()");
}
LOG.debug("Have read input token of size {} for processing by saslServer.evaluateResponse()",
saslToken.length);
saslToken = saslServer.evaluateResponse(saslToken);
return buildSaslResponse(
saslServer.isComplete() ? SaslState.SUCCESS : SaslState.CHALLENGE,
@ -2338,9 +2322,8 @@ public abstract class Server {
private RpcSaslProto buildSaslResponse(SaslState state, byte[] replyToken) {
if (LOG.isDebugEnabled()) {
LOG.debug("Will send " + state + " token of size "
+ ((replyToken != null) ? replyToken.length : null)
+ " from saslServer.");
LOG.debug("Will send {} token of size {} from saslServer.", state,
((replyToken != null) ? replyToken.length : null));
}
RpcSaslProto.Builder response = RpcSaslProto.newBuilder();
response.setState(state);
@ -2664,10 +2647,8 @@ public abstract class Server {
*/
private void unwrapPacketAndProcessRpcs(byte[] inBuf)
throws IOException, InterruptedException {
if (LOG.isDebugEnabled()) {
LOG.debug("Have read input token of size " + inBuf.length
+ " for processing by saslServer.unwrap()");
}
LOG.debug("Have read input token of size {} for processing by saslServer.unwrap()",
inBuf.length);
inBuf = saslServer.unwrap(inBuf, 0, inBuf.length);
ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(
inBuf));
@ -2729,9 +2710,7 @@ public abstract class Server {
getMessage(RpcRequestHeaderProto.getDefaultInstance(), buffer);
callId = header.getCallId();
retry = header.getRetryCount();
if (LOG.isDebugEnabled()) {
LOG.debug(" got #" + callId);
}
LOG.debug(" got #{}", callId);
checkRpcHeaders(header);
if (callId < 0) { // callIds typically used during connection setup
@ -2746,11 +2725,8 @@ public abstract class Server {
} catch (RpcServerException rse) {
// inform client of error, but do not rethrow else non-fatal
// exceptions will close connection!
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() +
": processOneRpc from client " + this +
" threw exception [" + rse + "]");
}
LOG.debug("{}: processOneRpc from client {} threw exception [{}]",
Thread.currentThread().getName(), this, rse);
// use the wrapped exception if there is one.
Throwable t = (rse.getCause() != null) ? rse.getCause() : rse;
final RpcCall call = new RpcCall(this, callId, retry);
@ -2962,9 +2938,7 @@ public abstract class Server {
ProxyUsers.authorize(user, this.getHostAddress());
}
authorize(user, protocolName, getHostInetAddress());
if (LOG.isDebugEnabled()) {
LOG.debug("Successfully authorized " + connectionContext);
}
LOG.debug("Successfully authorized {}.", connectionContext);
rpcMetrics.incrAuthorizationSuccesses();
} catch (AuthorizationException ae) {
LOG.info("Connection from " + this
@ -3081,7 +3055,7 @@ public abstract class Server {
@Override
public void run() {
LOG.debug(Thread.currentThread().getName() + ": starting");
LOG.debug("{}: starting", Thread.currentThread().getName());
SERVER.set(Server.this);
while (running) {
TraceScope traceScope = null;
@ -3115,9 +3089,7 @@ public abstract class Server {
call = null;
continue;
}
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
}
LOG.debug("{}: {} for RpcKind {}.", Thread.currentThread().getName(), call, call.rpcKind);
CurCall.set(call);
if (call.span != null) {
traceScope = tracer.activateSpan(call.span);
@ -3152,15 +3124,14 @@ public abstract class Server {
IOUtils.cleanupWithLogger(LOG, traceScope);
if (call != null) {
updateMetrics(call, startTimeNanos, connDropped);
ProcessingDetails.LOG.debug(
"Served: [{}]{} name={} user={} details={}",
ProcessingDetails.LOG.debug("Served: [{}]{} name={} user={} details={}",
call, (call.isResponseDeferred() ? ", deferred" : ""),
call.getDetailedMetricsName(), call.getRemoteUser(),
call.getProcessingDetails());
}
}
}
LOG.debug(Thread.currentThread().getName() + ": exiting");
LOG.debug("{}: exiting", Thread.currentThread().getName());
}
private void requeueCall(Call call)
@ -3389,14 +3360,13 @@ public abstract class Server {
" authentication requires a secret manager");
}
} else if (secretManager != null) {
LOG.debug(AuthenticationMethod.TOKEN +
" authentication enabled for secret manager");
LOG.debug("{} authentication enabled for secret manager", AuthenticationMethod.TOKEN);
// most preferred, go to the front of the line!
authMethods.add(AuthenticationMethod.TOKEN.getAuthMethod());
}
authMethods.add(confAuthenticationMethod.getAuthMethod());
LOG.debug("Server accepts auth methods:" + authMethods);
LOG.debug("Server accepts auth methods:{}", authMethods);
return authMethods;
}
@ -3556,9 +3526,7 @@ public abstract class Server {
synchronized (call.connection.saslServer) {
token = call.connection.saslServer.wrap(token, 0, token.length);
}
if (LOG.isDebugEnabled())
LOG.debug("Adding saslServer wrapped token of size " + token.length
+ " as call response.");
LOG.debug("Adding saslServer wrapped token of size {} as call response.", token.length);
// rebuild with sasl header and payload
RpcResponseHeaderProto saslHeader = RpcResponseHeaderProto.newBuilder()
.setCallId(AuthProtocol.SASL.callId)
@ -4004,11 +3972,8 @@ public abstract class Server {
Connection connection = new Connection(channel, Time.now(),
ingressPort, isOnAuxiliaryPort);
add(connection);
if (LOG.isDebugEnabled()) {
LOG.debug("Server connection from " + connection +
"; # active connections: " + size() +
"; # queued calls: " + callQueue.size());
}
LOG.debug("Server connection from {}; # active connections: {}; # queued calls: {}.",
connection, size(), callQueue.size());
return connection;
}
@ -4016,9 +3981,8 @@ public abstract class Server {
boolean exists = remove(connection);
if (exists) {
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() +
": disconnecting client " + connection +
". Number of active connections: "+ size());
LOG.debug("{}: disconnecting client {}. Number of active connections: {}.",
Thread.currentThread().getName(), connection, size());
}
// only close if actually removed to avoid double-closing due
// to possible races
@ -4080,9 +4044,7 @@ public abstract class Server {
if (!running) {
return;
}
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName()+": task running");
}
LOG.debug("{}: task running", Thread.currentThread().getName());
try {
closeIdle(false);
} finally {

View File

@ -522,8 +522,7 @@ public class NetworkTopology {
}
}
if (numOfDatanodes <= 0) {
LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\")."
+ " numOfDatanodes={}",
LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\"). numOfDatanodes={}",
scope, excludedScope, numOfDatanodes);
return null;
}
@ -539,10 +538,12 @@ public class NetworkTopology {
netlock.readLock().unlock();
}
}
LOG.debug("Choosing random from {} available nodes on node {},"
+ " scope={}, excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
availableNodes, innerNode, scope, excludedScope, excludedNodes,
numOfDatanodes);
if (LOG.isDebugEnabled()) {
LOG.debug("Choosing random from {} available nodes on node {}, scope={},"
+ " excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
availableNodes, innerNode, scope, excludedScope, excludedNodes,
numOfDatanodes);
}
Node ret = null;
if (availableNodes > 0) {
ret = chooseRandom(innerNode, node, excludedNodes, numOfDatanodes,

View File

@ -479,10 +479,9 @@ public class QuorumJournalManager implements JournalManager {
LOG.info("Successfully started new epoch " + loggers.getEpoch());
if (LOG.isDebugEnabled()) {
LOG.debug("newEpoch(" + loggers.getEpoch() + ") responses:\n" +
QuorumCall.mapToString(resps));
LOG.debug("newEpoch({}) responses:\n{}", loggers.getEpoch(), QuorumCall.mapToString(resps));
}
long mostRecentSegmentTxId = Long.MIN_VALUE;
for (NewEpochResponseProto r : resps.values()) {
if (r.hasLastSegmentTxId()) {
@ -518,10 +517,7 @@ public class QuorumJournalManager implements JournalManager {
// the cache used for RPC calls is not enabled; fall back to using the
// streaming mechanism to serve such requests
if (inProgressOk && inProgressTailingEnabled) {
if (LOG.isDebugEnabled()) {
LOG.debug("Tailing edits starting from txn ID " + fromTxnId +
" via RPC mechanism");
}
LOG.debug("Tailing edits starting from txn ID {} via RPC mechanism", fromTxnId);
try {
Collection<EditLogInputStream> rpcStreams = new ArrayList<>();
selectRpcInputStreams(rpcStreams, fromTxnId, onlyDurableTxns);
@ -585,8 +581,8 @@ public class QuorumJournalManager implements JournalManager {
int maxAllowedTxns = !onlyDurableTxns ? highestTxnCount :
responseCounts.get(responseCounts.size() - loggers.getMajoritySize());
if (maxAllowedTxns == 0) {
LOG.debug("No new edits available in logs; requested starting from " +
"ID {}", fromTxnId);
LOG.debug("No new edits available in logs; requested starting from ID {}",
fromTxnId);
return;
}
LogAction logAction = selectInputStreamLogHelper.record(fromTxnId);

View File

@ -1541,6 +1541,7 @@ public class BlockManager implements BlockStatsMXBean {
if (LOG.isDebugEnabled()) {
LOG.debug("blocks = {}", java.util.Arrays.asList(blocks));
}
final AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
LocatedBlockBuilder locatedBlocks = providedStorageMap
@ -1873,8 +1874,7 @@ public class BlockManager implements BlockStatsMXBean {
}
if (storage == null) {
blockLog.debug("BLOCK* findAndMarkBlockAsCorrupt: {} not found on {}",
blk, dn);
blockLog.debug("BLOCK* findAndMarkBlockAsCorrupt: {} not found on {}", blk, dn);
return;
}
markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock,
@ -1893,7 +1893,7 @@ public class BlockManager implements BlockStatsMXBean {
DatanodeStorageInfo storageInfo,
DatanodeDescriptor node) throws IOException {
if (b.getStored().isDeleted()) {
if(blockLog.isDebugEnabled()) {
if (blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
" corrupt as it does not belong to any file", b);
}
@ -1977,7 +1977,7 @@ public class BlockManager implements BlockStatsMXBean {
// Check how many copies we have of the block
if (nr.replicasOnStaleNodes() > 0 && !deleteCorruptReplicaImmediately) {
if(blockLog.isDebugEnabled()) {
if (blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* invalidateBlocks: postponing " +
"invalidation of {} on {} because {} replica(s) are located on " +
"nodes with potentially out-of-date block reports", b, dn,
@ -1990,8 +1990,7 @@ public class BlockManager implements BlockStatsMXBean {
// function and know there are enough live replicas, so we can delete it.
addToInvalidates(b.getCorrupted(), dn);
removeStoredBlock(b.getStored(), node);
blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.",
b, dn);
blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.", b, dn);
return true;
}
}
@ -2156,13 +2155,11 @@ public class BlockManager implements BlockStatsMXBean {
for (DatanodeStorageInfo target : targets) {
targetList.append(' ').append(target.getDatanodeDescriptor());
}
blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNodes(),
rw.getBlock(), targetList);
blockLog.debug("BLOCK* ask {} to replicate {} to {}",
rw.getSrcNodes(), rw.getBlock(), targetList);
}
}
blockLog.debug(
"BLOCK* neededReconstruction = {} pendingReconstruction = {}",
blockLog.debug("BLOCK* neededReconstruction = {} pendingReconstruction = {}",
neededReconstruction.size(), pendingReconstruction.size());
}
@ -2201,7 +2198,7 @@ public class BlockManager implements BlockStatsMXBean {
liveBlockIndices, liveBusyBlockIndices, excludeReconstructed, priority);
short requiredRedundancy = getExpectedLiveRedundancyNum(block,
numReplicas);
if(srcNodes == null || srcNodes.length == 0) {
if (srcNodes == null || srcNodes.length == 0) {
// block can not be reconstructed from any node
LOG.debug("Block {} cannot be reconstructed from any node", block);
NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
@ -2225,10 +2222,8 @@ public class BlockManager implements BlockStatsMXBean {
int pendingNum = pendingReconstruction.getNumReplicas(block);
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
neededReconstruction.remove(block, priority);
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
" it has enough replicas", block);
}
blockLog.debug("BLOCK* Removing {} from neededReconstruction as it has enough replicas",
block);
NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
return null;
}
@ -2328,10 +2323,8 @@ public class BlockManager implements BlockStatsMXBean {
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
neededReconstruction.remove(block, priority);
rw.resetTargets();
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
" it has enough replicas", block);
}
blockLog.debug("BLOCK* Removing {} from neededReconstruction as it has enough replicas",
block);
return false;
}
@ -2362,10 +2355,8 @@ public class BlockManager implements BlockStatsMXBean {
// The reason we use 'pending' is so we can retry
// reconstructions that fail after an appropriate amount of time.
pendingReconstruction.increment(block, targets);
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* block {} is moved from neededReconstruction to "
+ "pendingReconstruction", block);
}
blockLog.debug("BLOCK* block {} is moved from neededReconstruction to pendingReconstruction",
block);
int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
// remove from neededReconstruction
@ -2758,9 +2749,11 @@ public class BlockManager implements BlockStatsMXBean {
removeBlock(b);
}
if (trackBlockCounts) {
LOG.debug("Adjusting safe-mode totals for deletion."
+ "decreasing safeBlocks by {}, totalBlocks by {}",
numRemovedSafe, numRemovedComplete);
if (LOG.isDebugEnabled()) {
LOG.debug("Adjusting safe-mode totals for deletion."
+ "decreasing safeBlocks by {}, totalBlocks by {}",
numRemovedSafe, numRemovedComplete);
}
bmSafeMode.adjustBlockTotals(-numRemovedSafe, -numRemovedComplete);
}
}
@ -2913,13 +2906,11 @@ public class BlockManager implements BlockStatsMXBean {
namesystem.writeUnlock("processReport");
}
if(blockLog.isDebugEnabled()) {
if (blockLog.isDebugEnabled()) {
for (Block b : invalidatedBlocks) {
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* processReport 0x{} with lease ID 0x{}: {} on node {} size {} " +
"does not belong to any file.", strBlockReportId, fullBrLeaseId, b,
node, b.getNumBytes());
}
blockLog.debug("BLOCK* processReport 0x{} with lease ID 0x{}: {} on node {} size {} " +
"does not belong to any file.", strBlockReportId, fullBrLeaseId, b,
node, b.getNumBytes());
}
}
@ -2951,9 +2942,10 @@ public class BlockManager implements BlockStatsMXBean {
node.setLastBlockReportTime(now());
node.setLastBlockReportMonotonic(Time.monotonicNow());
}
LOG.debug("Processing RPC with index {} out of total {} RPCs in "
+ "processReport 0x{}", context.getCurRpc(),
context.getTotalRpcs(), Long.toHexString(context.getReportId()));
if (LOG.isDebugEnabled()) {
LOG.debug("Processing RPC with index {} out of total {} RPCs in processReport 0x{}",
context.getCurRpc(), context.getTotalRpcs(), Long.toHexString(context.getReportId()));
}
}
} finally {
namesystem.writeUnlock("removeBRLeaseIfNeeded");
@ -2978,14 +2970,16 @@ public class BlockManager implements BlockStatsMXBean {
BlockInfo bi = getStoredBlock(b);
if (bi == null) {
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
"Postponed mis-replicated block {} no longer found " +
"in block map.", b);
if (LOG.isDebugEnabled()) {
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
"Postponed mis-replicated block {} no longer found " +
"in block map.", b);
}
continue;
}
MisReplicationResult res = processMisReplicatedBlock(bi);
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
"Re-scanned block {}, result is {}", b, res);
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: Re-scanned block {}, result is {}",
b, res);
if (res == MisReplicationResult.POSTPONE) {
rescannedMisreplicatedBlocks.add(b);
}
@ -3077,7 +3071,7 @@ public class BlockManager implements BlockStatsMXBean {
}
}
if (isCorrupt) {
if(blockLog.isDebugEnabled()) {
if (blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
" {} on {} as corrupt because the dn is not in the new committed " +
"storage list.", b, storage.getDatanodeDescriptor());
@ -3113,6 +3107,7 @@ public class BlockManager implements BlockStatsMXBean {
iblk.getBlockName(), storageInfo.getDatanodeDescriptor(),
iblk.getNumBytes(), reportedState);
}
if (shouldPostponeBlocksFromFuture && isGenStampInFuture(iblk)) {
queueReportedBlock(storageInfo, iblk, reportedState,
QUEUE_REASON_FUTURE_GENSTAMP);
@ -3329,9 +3324,11 @@ public class BlockManager implements BlockStatsMXBean {
ReplicaState reportedState, String reason) {
assert shouldPostponeBlocksFromFuture;
LOG.debug("Queueing reported block {} in state {}" +
" from datanode {} for later processing because {}.",
block, reportedState, storageInfo.getDatanodeDescriptor(), reason);
if (LOG.isDebugEnabled()) {
LOG.debug("Queueing reported block {} in state {}" +
" from datanode {} for later processing because {}.",
block, reportedState, storageInfo.getDatanodeDescriptor(), reason);
}
pendingDNMessages.enqueueReportedBlock(storageInfo, block, reportedState);
}
@ -3598,11 +3595,8 @@ public class BlockManager implements BlockStatsMXBean {
}
if (storedBlock == null || storedBlock.isDeleted()) {
// If this block does not belong to anyfile, then we are done.
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not" +
" belong to any file", block, node, block.getNumBytes());
}
blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not belong to any file",
block, node, block.getNumBytes());
// we could add this block to invalidate set of this datanode.
// it will happen in next block report otherwise.
return block;
@ -3630,7 +3624,7 @@ public class BlockManager implements BlockStatsMXBean {
corruptReplicas.removeFromCorruptReplicasMap(block, node,
Reason.GENSTAMP_MISMATCH);
curReplicaDelta = 0;
if(blockLog.isDebugEnabled()) {
if (blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* addStoredBlock: Redundant addStoredBlock request"
+ " received for {} on node {} size {}", storedBlock, node,
storedBlock.getNumBytes());
@ -3735,10 +3729,8 @@ public class BlockManager implements BlockStatsMXBean {
removedFromBlocksMap = false;
}
} catch (IOException e) {
if(blockLog.isDebugEnabled()) {
blockLog.debug("invalidateCorruptReplicas error in deleting bad block"
+ " {} on {}", blk, node, e);
}
blockLog.debug("invalidateCorruptReplicas error in deleting bad block {} on {}",
blk, node, e);
removedFromBlocksMap = false;
}
}
@ -3920,8 +3912,8 @@ public class BlockManager implements BlockStatsMXBean {
BlockInfo blk = iter.next();
MisReplicationResult r = processMisReplicatedBlock(blk);
processed++;
LOG.debug("BLOCK* processMisReplicatedBlocks: " +
"Re-scanned block {}, result is {}", blk, r);
LOG.debug("BLOCK* processMisReplicatedBlocks: Re-scanned block {}, result is {}",
blk, r);
}
} finally {
namesystem.writeUnlock("processMisReplicatedBlocks");
@ -4187,10 +4179,8 @@ public class BlockManager implements BlockStatsMXBean {
//
final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen);
addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor());
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* chooseExcessRedundancies: "
+ "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
}
blockLog.debug("BLOCK* chooseExcessRedundancies: ({}, {}) is added to invalidated blocks set",
chosen, storedBlock);
}
private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block,
@ -4212,8 +4202,8 @@ public class BlockManager implements BlockStatsMXBean {
assert (namesystem.hasWriteLock());
{
if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
" removed from node {}", storedBlock, node);
blockLog.debug("BLOCK* removeStoredBlock: {} has already been removed from node {}",
storedBlock, node);
return;
}
@ -4225,8 +4215,10 @@ public class BlockManager implements BlockStatsMXBean {
removed |= node.getCached().remove(cblock);
removed |= node.getPendingUncached().remove(cblock);
if (removed) {
blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
+ "related lists on node {}", storedBlock, node);
if (blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
+ "related lists on node {}", storedBlock, node);
}
}
}
@ -4251,8 +4243,9 @@ public class BlockManager implements BlockStatsMXBean {
for (ReplicaUnderConstruction r : staleReplicas) {
removeStoredBlock(block,
r.getExpectedStorageLocation().getDatanodeDescriptor());
blockLog.debug("BLOCK* Removing stale replica {} of {}", r,
Block.toString(r));
if (blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* Removing stale replica {} of {}", r, Block.toString(r));
}
}
}
/**
@ -4380,10 +4373,8 @@ public class BlockManager implements BlockStatsMXBean {
maxNumBlocksToLog, numBlocksLogged);
}
for (Block b : toInvalidate) {
if(blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not " +
"belong to any file", b, node, b.getNumBytes());
}
blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not belong to any file",
b, node, b.getNumBytes());
addToInvalidates(b, node);
}
for (BlockToMarkCorrupt b : toCorrupt) {
@ -4464,7 +4455,7 @@ public class BlockManager implements BlockStatsMXBean {
blockLog.debug("BLOCK* block {}: {} is received from {}",
rdbi.getStatus(), rdbi.getBlock(), node);
}
if(blockLog.isDebugEnabled()) {
if (blockLog.isDebugEnabled()) {
blockLog.debug("*BLOCK* NameNode.processIncrementalBlockReport: from "
+ "{} receiving: {}, received: {}, deleted: {}", node, receiving,
received, deleted);
@ -4843,8 +4834,10 @@ public class BlockManager implements BlockStatsMXBean {
} finally {
namesystem.writeUnlock("invalidateWorkForOneNode");
}
blockLog.debug("BLOCK* {}: ask {} to delete {}", getClass().getSimpleName(),
dn, toInvalidate);
if (blockLog.isDebugEnabled()) {
blockLog.debug("BLOCK* {}: ask {} to delete {}",
getClass().getSimpleName(), dn, toInvalidate);
}
return toInvalidate.size();
}
@ -5110,8 +5103,8 @@ public class BlockManager implements BlockStatsMXBean {
}
}
if (isSleep) {
LOG.debug("Clear markedDeleteQueue over {}" +
" millisecond to release the write lock", deleteBlockLockTimeMs);
LOG.debug("Clear markedDeleteQueue over {} millisecond to release the write lock",
deleteBlockLockTimeMs);
}
try {
Thread.sleep(deleteBlockUnlockIntervalTimeMs);

View File

@ -238,8 +238,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
return getPipeline(writer,
results.toArray(new DatanodeStorageInfo[results.size()]));
} catch (NotEnoughReplicasException nr) {
LOG.debug("Failed to choose with favored nodes (={}), disregard favored"
+ " nodes hint and retry.", favoredNodes, nr);
LOG.debug("Failed to choose with favored nodes (={}), disregard favored nodes hint and retry",
favoredNodes, nr);
// Fall back to regular block placement disregarding favored nodes hint
return chooseTarget(src, numOfReplicas, writer,
new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
@ -715,17 +715,19 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor();
if (nextNode != localMachine) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to choose from local rack (location = " + localRack
+ "), retry with the rack of the next replica (location = "
+ nextNode.getNetworkLocation() + ")", e);
LOG.debug("Failed to choose from local rack (location = {}), retry with the rack "
+ "of the next replica (location = {})", localRack,
nextNode.getNetworkLocation(), e);
}
return chooseFromNextRack(nextNode, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageTypes);
}
}
LOG.debug("Failed to choose from local rack (location = {}); the second"
+ " replica is not found, retry choosing randomly", localRack, e);
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to choose from local rack (location = {}); the second"
+ " replica is not found, retry choosing randomly", localRack, e);
}
//the second replica is not found, randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
@ -745,9 +747,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
return chooseRandom(nextRack, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageTypes);
} catch (NotEnoughReplicasException e) {
LOG.debug("Failed to choose from the next rack (location = {}), "
+ "retry choosing randomly", nextRack, e);
// otherwise randomly choose one from the network
LOG.debug("Failed to choose from the next rack (location = {}), retry choosing randomly",
nextRack, e);
// otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageTypes);
}
@ -775,10 +778,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
excludedNodes, blocksize, maxReplicasPerRack, results,
avoidStaleNodes, storageTypes);
} catch (NotEnoughReplicasException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to choose remote rack (location = ~"
+ localMachine.getNetworkLocation() + "), fallback to local rack", e);
}
LOG.debug("Failed to choose remote rack (location = ~{}), fallback to local rack",
localMachine.getNetworkLocation(), e);
chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
localMachine.getNetworkLocation(), excludedNodes, blocksize,
maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
@ -1276,8 +1277,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
firstOne = false;
if (cur == null) {
LOG.debug(
"No excess replica can be found. excessTypes: {}. "
+ "moreThanOne: {}. exactlyOne: {}.",
"No excess replica can be found. excessTypes: {}. moreThanOne: {}. exactlyOne: {}.",
excessTypes, moreThanOne, exactlyOne);
break;
}

View File

@ -152,10 +152,12 @@ public class DatanodeAdminManager {
executor.scheduleWithFixedDelay(monitor, intervalSecs, intervalSecs,
TimeUnit.SECONDS);
LOG.debug("Activating DatanodeAdminManager with interval {} seconds, " +
"{} max blocks per interval, " +
"{} max concurrently tracked nodes.", intervalSecs,
blocksPerInterval, maxConcurrentTrackedNodes);
if (LOG.isDebugEnabled()) {
LOG.debug("Activating DatanodeAdminManager with interval {} seconds, " +
"{} max blocks per interval, " +
"{} max concurrently tracked nodes.", intervalSecs,
blocksPerInterval, maxConcurrentTrackedNodes);
}
}
/**

View File

@ -842,9 +842,7 @@ public class DatanodeManager {
decrementVersionCount(nodeInfo.getSoftwareVersion());
blockManager.getBlockReportLeaseManager().unregister(nodeInfo);
if (LOG.isDebugEnabled()) {
LOG.debug("remove datanode " + nodeInfo);
}
LOG.debug("remove datanode {}.", nodeInfo);
blockManager.checkSafeMode();
}
@ -906,8 +904,8 @@ public class DatanodeManager {
resolveUpgradeDomain(node);
if (LOG.isDebugEnabled()) {
LOG.debug(getClass().getSimpleName() + ".addDatanode: "
+ "node " + node + " is added to datanodeMap.");
LOG.debug("{}.addDatanode: node {} is added to datanodeMap.",
getClass().getSimpleName(), node);
}
}
@ -918,9 +916,8 @@ public class DatanodeManager {
host2DatanodeMap.remove(datanodeMap.remove(key));
}
if (LOG.isDebugEnabled()) {
LOG.debug(getClass().getSimpleName() + ".wipeDatanode("
+ node + "): storage " + key
+ " is removed from datanodeMap.");
LOG.debug("{}.wipeDatanode({}): storage {} is removed from datanodeMap.",
getClass().getSimpleName(), node, key);
}
}
@ -1189,10 +1186,7 @@ public class DatanodeManager {
// The same datanode has been just restarted to serve the same data
// storage. We do not need to remove old data blocks, the delta will
// be calculated on the next block report from the datanode
if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* registerDatanode: "
+ "node restarted.");
}
NameNode.stateChangeLog.debug("BLOCK* registerDatanode: node restarted.");
} else {
// nodeS is found
/* The registering datanode is a replacement node for the existing
@ -1535,9 +1529,11 @@ public class DatanodeManager {
"now be replicated cross-rack";
LOG.info(message);
} else {
message += "Not checking for mis-replicated blocks because this NN is " +
"not yet processing repl queues.";
LOG.debug(message);
if (LOG.isDebugEnabled()) {
message += "Not checking for mis-replicated blocks because this NN "
+ "is not yet processing repl queues.";
LOG.debug(message);
}
}
hasClusterEverBeenMultiRack = true;
if (blockManager.isPopulatingReplQueues()) {
@ -1659,11 +1655,9 @@ public class DatanodeManager {
}
if (LOG.isDebugEnabled()) {
LOG.debug("getDatanodeListForReport with " +
"includedNodes = " + hostConfigManager.getIncludes() +
", excludedNodes = " + hostConfigManager.getExcludes() +
", foundNodes = " + foundNodes +
", nodes = " + nodes);
LOG.debug("getDatanodeListForReport with includedNodes = {}, excludedNodes = {}"
+ ", foundNodes = {}, nodes = {}.", hostConfigManager.getIncludes(),
hostConfigManager.getExcludes(), foundNodes, nodes);
}
return nodes;
}
@ -1847,10 +1841,8 @@ public class DatanodeManager {
(double) (totalReplicateBlocks * maxTransfers) / totalBlocks);
int numECTasks = (int) Math.ceil(
(double) (totalECBlocks * maxTransfers) / totalBlocks);
if (LOG.isDebugEnabled()) {
LOG.debug("Pending replication tasks: " + numReplicationTasks
+ " erasure-coded tasks: " + numECTasks);
}
LOG.debug("Pending replication tasks: {} erasure-coded tasks: {}.",
numReplicationTasks, numECTasks);
// check pending replication tasks
List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
numReplicationTasks);
@ -1906,9 +1898,7 @@ public class DatanodeManager {
if (slowPeerTracker.isSlowPeerTrackerEnabled()) {
final Map<String, OutlierMetrics> slowPeersMap = slowPeers.getSlowPeers();
if (!slowPeersMap.isEmpty()) {
if (LOG.isDebugEnabled()) {
LOG.debug("DataNode " + nodeReg + " reported slow peers: " + slowPeersMap);
}
LOG.debug("DataNode {} reported slow peers: {}.", nodeReg, slowPeersMap);
for (Map.Entry<String, OutlierMetrics> slowNodeEntry : slowPeersMap.entrySet()) {
slowPeerTracker.addReport(slowNodeEntry.getKey(), nodeReg.getIpcAddr(false),
slowNodeEntry.getValue());
@ -1918,10 +1908,7 @@ public class DatanodeManager {
if (slowDiskTracker != null) {
if (!slowDisks.getSlowDisks().isEmpty()) {
if (LOG.isDebugEnabled()) {
LOG.debug("DataNode " + nodeReg + " reported slow disks: " +
slowDisks.getSlowDisks());
}
LOG.debug("DataNode {} reported slow disks: {}.", nodeReg, slowDisks.getSlowDisks());
slowDiskTracker.addSlowDiskReport(nodeReg.getIpcAddr(false), slowDisks);
}
slowDiskTracker.checkAndUpdateReportIfNecessary();
@ -1950,9 +1937,7 @@ public class DatanodeManager {
StorageReport[] reports, long cacheCapacity,
long cacheUsed, int xceiverCount, int failedVolumes,
VolumeFailureSummary volumeFailureSummary) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Received handleLifeline from nodeReg = " + nodeReg);
}
LOG.debug("Received handleLifeline from nodeReg = {}.", nodeReg);
DatanodeDescriptor nodeinfo = getDatanode(nodeReg);
if (nodeinfo == null || !nodeinfo.isRegistered()) {
// This can happen if the lifeline message comes when DataNode is either

View File

@ -976,9 +976,8 @@ public class CacheManager {
public final void processCacheReport(final DatanodeID datanodeID,
final List<Long> blockIds) throws IOException {
if (!enabled) {
LOG.debug("Ignoring cache report from {} because {} = false. " +
"number of blocks: {}", datanodeID,
DFS_NAMENODE_CACHING_ENABLED_KEY, blockIds.size());
LOG.debug("Ignoring cache report from {} because {} = false. number of blocks: {}",
datanodeID, DFS_NAMENODE_CACHING_ENABLED_KEY, blockIds.size());
return;
}
namesystem.writeLock();
@ -1003,9 +1002,8 @@ public class CacheManager {
if (metrics != null) {
metrics.addCacheBlockReport((int) (endTime - startTime));
}
LOG.debug("Processed cache report from {}, blocks: {}, " +
"processing time: {} msecs", datanodeID, blockIds.size(),
(endTime - startTime));
LOG.debug("Processed cache report from {}, blocks: {}, processing time: {} msecs",
datanodeID, blockIds.size(), (endTime - startTime));
}
private void processCacheReportImpl(final DatanodeDescriptor datanode,

View File

@ -649,7 +649,7 @@ public class FSImage implements Closeable {
*/
void reloadFromImageFile(File file, FSNamesystem target) throws IOException {
target.clear();
LOG.debug("Reloading namespace from " + file);
LOG.debug("Reloading namespace from {}.", file);
loadFSImage(file, target, null, false);
}
@ -728,7 +728,7 @@ public class FSImage implements Closeable {
}
for (EditLogInputStream l : editStreams) {
LOG.debug("Planning to load edit log stream: " + l);
LOG.debug("Planning to load edit log stream: {}.", l);
}
if (!editStreams.iterator().hasNext()) {
LOG.info("No edit log streams selected.");
@ -892,8 +892,10 @@ public class FSImage implements Closeable {
FSNamesystem target, long maxTxnsToRead,
StartupOption startOpt, MetaRecoveryContext recovery)
throws IOException {
LOG.debug("About to load edits:\n " + Joiner.on("\n ").join(editStreams));
if (LOG.isDebugEnabled()) {
LOG.debug("About to load edits:\n {}.", Joiner.on("\n ").join(editStreams));
}
long prevLastAppliedTxId = lastAppliedTxId;
long remainingReadTxns = maxTxnsToRead;
try {
@ -1348,10 +1350,10 @@ public class FSImage implements Closeable {
final File fromFile = NNStorage.getStorageFile(sd, fromNnf, txid);
final File toFile = NNStorage.getStorageFile(sd, toNnf, txid);
// renameTo fails on Windows if the destination file already exists.
if(LOG.isDebugEnabled()) {
LOG.debug("renaming " + fromFile.getAbsolutePath()
+ " to " + toFile.getAbsolutePath());
if (LOG.isDebugEnabled()) {
LOG.debug("renaming {} to {}", fromFile.getAbsoluteFile(), toFile.getAbsolutePath());
}
if (!fromFile.renameTo(toFile)) {
if (!toFile.delete() || !fromFile.renameTo(toFile)) {
throw new IOException("renaming " + fromFile.getAbsolutePath() + " to " +

View File

@ -1407,8 +1407,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
if (LOG.isDebugEnabled()) {
LOG.debug("NameNode metadata after re-processing " +
"replication and invalidation queues during failover:\n" +
LOG.debug("NameNode metadata after re-processing {}"
+ "replication and invalidation queues during failover:\n",
metaSaveAsString());
}
@ -2675,8 +2675,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
for (CryptoProtocolVersion c : supportedVersions) {
if (c.equals(CryptoProtocolVersion.UNKNOWN)) {
LOG.debug("Ignoring unknown CryptoProtocolVersion provided by " +
"client: {}", c.getUnknownValue());
LOG.debug("Ignoring unknown CryptoProtocolVersion provided by client: {}",
c.getUnknownValue());
continue;
}
if (c.equals(required)) {
@ -2987,8 +2987,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
requireEffectiveLayoutVersionForFeature(Feature.APPEND_NEW_BLOCK);
}
NameNode.stateChangeLog.debug(
"DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}",
NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}",
srcArg, holder, clientMachine);
try {
boolean skipSync = false;
@ -3045,8 +3044,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
DatanodeInfo[] excludedNodes, String[] favoredNodes,
EnumSet<AddBlockFlag> flags) throws IOException {
final String operationName = "getAdditionalBlock";
NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {} inodeId {}" +
" for {}", src, fileId, clientName);
NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {} inodeId {} for {}",
src, fileId, clientName);
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
FSDirWriteFileOp.ValidateAddBlockResult r;
@ -3148,8 +3147,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
*/
void abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
throws IOException {
NameNode.stateChangeLog.debug(
"BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
checkOperation(OperationCategory.WRITE);
final FSPermissionChecker pc = getPermissionChecker();
FSPermissionChecker.setOperationType(null);
@ -3158,8 +3156,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
checkOperation(OperationCategory.WRITE);
checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
FSDirWriteFileOp.abandonBlock(dir, pc, b, fileId, src, holder);
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: {} is " +
"removed from pendingCreates", b);
NameNode.stateChangeLog.debug(
"BLOCK* NameSystem.abandonBlock: {} is removed from pendingCreates", b);
} finally {
writeUnlock("abandonBlock");
}
@ -4016,9 +4014,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
if ((!iFile.isUnderConstruction() || storedBlock.isComplete()) &&
iFile.getLastBlock().isComplete()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Unexpected block (={}) since the file (={}) is not "
+ "under construction", oldBlock, iFile.getLocalName());
LOG.debug("Unexpected block (={}) since the file (={}) is not under construction",
oldBlock, iFile.getLocalName());
}
return;
}
@ -4478,8 +4477,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
assert hasWriteLock();
// file is closed
getEditLog().logCloseFile(path, file);
NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted" +
" to the file system", path, file.getBlocks().length);
NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted to the file system",
path, file.getBlocks().length);
}
/**
@ -6107,9 +6106,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
if (cookieTab[0] == null) {
cookieTab[0] = String.valueOf(getIntCookie(cookieTab[0]));
}
if (LOG.isDebugEnabled()) {
LOG.debug("there are no corrupt file blocks.");
}
LOG.debug("there are no corrupt file blocks.");
return corruptFiles;
}

View File

@ -810,10 +810,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
throws IOException {
checkNNStartup();
String clientMachine = getClientMachine();
if (stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* NameNode.create: file "
+src+" for "+clientName+" at "+clientMachine);
}
stateChangeLog.debug("*DIR* NameNode.create: file {} for {} at {}.",
src, clientName, clientMachine);
if (!checkPathLength(src)) {
throw new IOException("create: Pathname too long. Limit "
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@ -845,10 +843,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
EnumSetWritable<CreateFlag> flag) throws IOException {
checkNNStartup();
String clientMachine = getClientMachine();
if (stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* NameNode.append: file "
+src+" for "+clientName+" at "+clientMachine);
}
stateChangeLog.debug("*DIR* NameNode.append: file {} for {} at {}.",
src, clientName, clientMachine);
namesystem.checkOperation(OperationCategory.WRITE);
CacheEntryWithPayload cacheEntry = getCacheEntryWithPayload(null);
if (cacheEntry != null && cacheEntry.isSuccess()) {
@ -894,8 +890,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
public void setStoragePolicy(String src, String policyName)
throws IOException {
checkNNStartup();
stateChangeLog.debug("*DIR* NameNode.setStoragePolicy for path: {}, " +
"policyName: {}", src, policyName);
stateChangeLog.debug("*DIR* NameNode.setStoragePolicy for path: {}, policyName: {}",
src, policyName);
namesystem.setStoragePolicy(src, policyName);
}
@ -949,13 +945,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
) throws IOException {
checkNNStartup();
if (LOG.isDebugEnabled()) {
LOG.debug("getAdditionalDatanode: src=" + src
+ ", fileId=" + fileId
+ ", blk=" + blk
+ ", existings=" + Arrays.asList(existings)
+ ", excludes=" + Arrays.asList(excludes)
+ ", numAdditionalNodes=" + numAdditionalNodes
+ ", clientName=" + clientName);
LOG.debug("getAdditionalDatanode: src={}, fileId={}, blk={}, existings={}, excludes={}"
+ ", numAdditionalNodes={}, clientName={}", src, fileId, blk, Arrays.asList(existings),
Arrays.asList(excludes), numAdditionalNodes, clientName);
}
metrics.incrGetAdditionalDatanodeOps();
@ -1053,9 +1045,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
@Override // ClientProtocol
public boolean rename(String src, String dst) throws IOException {
checkNNStartup();
if(stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
}
stateChangeLog.debug("*DIR* NameNode.rename: {} to {}.", src, dst);
if (!checkPathLength(dst)) {
throw new IOException("rename: Pathname too long. Limit "
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@ -1081,8 +1071,10 @@ public class NameNodeRpcServer implements NamenodeProtocols {
@Override // ClientProtocol
public void concat(String trg, String[] src) throws IOException {
checkNNStartup();
stateChangeLog.debug("*DIR* NameNode.concat: src path {} to" +
" target path {}", Arrays.toString(src), trg);
if (stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* NameNode.concat: src path {} to target path {}",
Arrays.toString(src), trg);
}
namesystem.checkOperation(OperationCategory.WRITE);
CacheEntry cacheEntry = getCacheEntry();
if (cacheEntry != null && cacheEntry.isSuccess()) {
@ -1102,9 +1094,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
public void rename2(String src, String dst, Options.Rename... options)
throws IOException {
checkNNStartup();
if(stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
}
stateChangeLog.debug("*DIR* NameNode.rename: {} to {}.", src, dst);
if (!checkPathLength(dst)) {
throw new IOException("rename: Pathname too long. Limit "
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@ -1128,8 +1118,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
public boolean truncate(String src, long newLength, String clientName)
throws IOException {
checkNNStartup();
stateChangeLog
.debug("*DIR* NameNode.truncate: " + src + " to " + newLength);
stateChangeLog.debug("*DIR* NameNode.truncate: {} to {}", src, newLength);
String clientMachine = getClientMachine();
try {
return namesystem.truncate(
@ -1142,10 +1131,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
@Override // ClientProtocol
public boolean delete(String src, boolean recursive) throws IOException {
checkNNStartup();
if (stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* Namenode.delete: src=" + src
+ ", recursive=" + recursive);
}
stateChangeLog.debug("*DIR* Namenode.delete: src={}, recursive={}.", src, recursive);
namesystem.checkOperation(OperationCategory.WRITE);
CacheEntry cacheEntry = getCacheEntry();
if (cacheEntry != null && cacheEntry.isSuccess()) {
@ -1178,9 +1164,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
throws IOException {
checkNNStartup();
if(stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src);
}
stateChangeLog.debug("*DIR* NameNode.mkdirs: {}.", src);
if (!checkPathLength(src)) {
throw new IOException("mkdirs: Pathname too long. Limit "
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@ -1649,10 +1633,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
final BlockReportContext context) throws IOException {
checkNNStartup();
verifyRequest(nodeReg);
if(blockStateChangeLog.isDebugEnabled()) {
blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
+ "from " + nodeReg + ", reports.length=" + reports.length);
}
blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: from {}, reports.length={}.",
nodeReg, reports.length);
final BlockManager bm = namesystem.getBlockManager();
boolean noStaleStorages = false;
try {
@ -1695,10 +1677,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
String poolId, List<Long> blockIds) throws IOException {
checkNNStartup();
verifyRequest(nodeReg);
if (blockStateChangeLog.isDebugEnabled()) {
blockStateChangeLog.debug("*BLOCK* NameNode.cacheReport: "
+ "from " + nodeReg + " " + blockIds.size() + " blocks");
}
blockStateChangeLog.debug("*BLOCK* NameNode.cacheReport: from {} {} blocks",
nodeReg, blockIds.size());
namesystem.getCacheManager().processCacheReport(nodeReg, blockIds);
return null;
}
@ -1710,11 +1690,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
checkNNStartup();
verifyRequest(nodeReg);
metrics.incrBlockReceivedAndDeletedOps();
if(blockStateChangeLog.isDebugEnabled()) {
blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
+"from "+nodeReg+" "+receivedAndDeletedBlocks.length
+" blocks.");
}
blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: from {} {} blocks.",
nodeReg, receivedAndDeletedBlocks.length);
final BlockManager bm = namesystem.getBlockManager();
for (final StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) {
bm.enqueueBlockOp(new Runnable() {
@ -1842,9 +1819,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
@Override // GetUserMappingsProtocol
public String[] getGroupsForUser(String user) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Getting groups for user " + user);
}
LOG.debug("Getting groups for user {}", user);
return UserGroupInformation.createRemoteUser(user).getGroupNames();
}
@ -2022,9 +1997,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
public void renameSnapshot(String snapshotRoot, String snapshotOldName,
String snapshotNewName) throws IOException {
checkNNStartup();
LOG.debug("*DIR* NameNode.renameSnapshot: Snapshot Path {}, " +
"snapshotOldName {}, snapshotNewName {}", snapshotRoot,
snapshotOldName, snapshotNewName);
LOG.debug(
"*DIR* NameNode.renameSnapshot: Snapshot Path {},snapshotOldName {}, snapshotNewName {}",
snapshotRoot, snapshotOldName, snapshotNewName);
if (snapshotNewName == null || snapshotNewName.isEmpty()) {
throw new IOException("The new snapshot name is null or empty.");
}
@ -2318,8 +2293,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
try {
if (ecPolicyName == null) {
ecPolicyName = defaultECPolicyName;
LOG.debug("No policy name is specified, " +
"set the default policy name instead");
LOG.debug("No policy name is specified, set the default policy name instead");
}
LOG.debug("Set erasure coding policy {} on {}", ecPolicyName, src);
namesystem.setErasureCodingPolicy(src, ecPolicyName, cacheEntry != null);

View File

@ -262,9 +262,7 @@ public class EditLogTailer {
nnCount = nns.size();
// setup the iterator to endlessly loop the nns
this.nnLookup = Iterators.cycle(nns);
LOG.debug("logRollPeriodMs=" + logRollPeriodMs +
" sleepTime=" + sleepTimeMs);
LOG.debug("logRollPeriodMs={} sleepTime={}.", logRollPeriodMs, sleepTimeMs);
}
public void start() {
@ -360,9 +358,7 @@ public class EditLogTailer {
currentLastTxnId, lastTxnId);
return 0;
}
if (LOG.isDebugEnabled()) {
LOG.debug("edit streams to load from: " + streams.size());
}
LOG.debug("edit streams to load from: {}.", streams.size());
// Once we have streams to load, errors encountered are legitimate cause
// for concern, so we don't catch them here. Simple errors reading from
@ -375,10 +371,7 @@ public class EditLogTailer {
editsLoaded = elie.getNumEditsLoaded();
throw elie;
} finally {
if (editsLoaded > 0 || LOG.isDebugEnabled()) {
LOG.debug(String.format("Loaded %d edits starting from txid %d ",
editsLoaded, lastTxnId));
}
LOG.debug("Loaded {} edits starting from txid {}.", editsLoaded, lastTxnId);
NameNode.getNameNodeMetrics().addNumEditLogLoaded(editsLoaded);
}