HDFS-16648. Add isDebugEnabled check for debug blockLogs in some classes (#4529)
This commit is contained in:
parent
bd0f9a46e1
commit
25ccdc77af
|
@ -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 {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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 " +
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue