Skip to content

Commit 25ccdc7

Browse files
authored
HDFS-16648. Add isDebugEnabled check for debug blockLogs in some classes (#4529)
1 parent bd0f9a4 commit 25ccdc7

File tree

12 files changed

+218
-315
lines changed

12 files changed

+218
-315
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

Lines changed: 35 additions & 73 deletions
Original file line numberDiff line numberDiff line change
@@ -287,11 +287,8 @@ public static void registerProtocolEngine(RPC.RpcKind rpcKind,
287287
throw new IllegalArgumentException("ReRegistration of rpcKind: " +
288288
rpcKind);
289289
}
290-
if (LOG.isDebugEnabled()) {
291-
LOG.debug("rpcKind=" + rpcKind +
292-
", rpcRequestWrapperClass=" + rpcRequestWrapperClass +
293-
", rpcInvoker=" + rpcInvoker);
294-
}
290+
LOG.debug("rpcKind={}, rpcRequestWrapperClass={}, rpcInvoker={}.",
291+
rpcKind, rpcRequestWrapperClass, rpcInvoker);
295292
}
296293

297294
public Class<? extends Writable> getRpcRequestWrapper(
@@ -1212,9 +1209,7 @@ public Void run() throws Exception {
12121209
deltaNanos = Time.monotonicNowNanos() - startNanos;
12131210
details.set(Timing.RESPONSE, deltaNanos, TimeUnit.NANOSECONDS);
12141211
} else {
1215-
if (LOG.isDebugEnabled()) {
1216-
LOG.debug("Deferring response for callId: " + this.callId);
1217-
}
1212+
LOG.debug("Deferring response for callId: {}", this.callId);
12181213
}
12191214
return null;
12201215
}
@@ -1711,9 +1706,7 @@ private void doRunLoop() {
17111706
// If there were some calls that have not been sent out for a
17121707
// long time, discard them.
17131708
//
1714-
if(LOG.isDebugEnabled()) {
1715-
LOG.debug("Checking for old call responses.");
1716-
}
1709+
LOG.debug("Checking for old call responses.");
17171710
ArrayList<RpcCall> calls;
17181711

17191712
// get the list of channels from list of keys.
@@ -1813,9 +1806,8 @@ private boolean processResponse(LinkedList<RpcCall> responseQueue,
18131806
//
18141807
call = responseQueue.removeFirst();
18151808
SocketChannel channel = call.connection.channel;
1816-
if (LOG.isDebugEnabled()) {
1817-
LOG.debug(Thread.currentThread().getName() + ": responding to " + call);
1818-
}
1809+
1810+
LOG.debug("{}: responding to {}.", Thread.currentThread().getName(), call);
18191811
//
18201812
// Send as much data as we can in the non-blocking fashion
18211813
//
@@ -1832,10 +1824,8 @@ private boolean processResponse(LinkedList<RpcCall> responseQueue,
18321824
} else {
18331825
done = false; // more calls pending to be sent.
18341826
}
1835-
if (LOG.isDebugEnabled()) {
1836-
LOG.debug(Thread.currentThread().getName() + ": responding to " + call
1837-
+ " Wrote " + numBytes + " bytes.");
1838-
}
1827+
LOG.debug("{}: responding to {} Wrote {} bytes.",
1828+
Thread.currentThread().getName(), call, numBytes);
18391829
} else {
18401830
//
18411831
// If we were unable to write the entire response out, then
@@ -1860,10 +1850,8 @@ private boolean processResponse(LinkedList<RpcCall> responseQueue,
18601850
decPending();
18611851
}
18621852
}
1863-
if (LOG.isDebugEnabled()) {
1864-
LOG.debug(Thread.currentThread().getName() + ": responding to " + call
1865-
+ " Wrote partial " + numBytes + " bytes.");
1866-
}
1853+
LOG.debug("{}: responding to {} Wrote partial {} bytes.",
1854+
Thread.currentThread().getName(), call, numBytes);
18671855
}
18681856
error = false; // everything went off well
18691857
}
@@ -2209,13 +2197,11 @@ private void saslProcess(RpcSaslProto saslMessage)
22092197

22102198
if (saslServer != null && saslServer.isComplete()) {
22112199
if (LOG.isDebugEnabled()) {
2212-
LOG.debug("SASL server context established. Negotiated QoP is "
2213-
+ saslServer.getNegotiatedProperty(Sasl.QOP));
2200+
LOG.debug("SASL server context established. Negotiated QoP is {}.",
2201+
saslServer.getNegotiatedProperty(Sasl.QOP));
22142202
}
22152203
user = getAuthorizedUgi(saslServer.getAuthorizationID());
2216-
if (LOG.isDebugEnabled()) {
2217-
LOG.debug("SASL server successfully authenticated client: " + user);
2218-
}
2204+
LOG.debug("SASL server successfully authenticated client: {}.", user);
22192205
rpcMetrics.incrAuthenticationSuccesses();
22202206
AUDITLOG.info(AUTH_SUCCESSFUL_FOR + user + " from " + toString());
22212207
saslContextEstablished = true;
@@ -2320,10 +2306,8 @@ private RpcSaslProto processSaslToken(RpcSaslProto saslMessage)
23202306
throw new SaslException("Client did not send a token");
23212307
}
23222308
byte[] saslToken = saslMessage.getToken().toByteArray();
2323-
if (LOG.isDebugEnabled()) {
2324-
LOG.debug("Have read input token of size " + saslToken.length
2325-
+ " for processing by saslServer.evaluateResponse()");
2326-
}
2309+
LOG.debug("Have read input token of size {} for processing by saslServer.evaluateResponse()",
2310+
saslToken.length);
23272311
saslToken = saslServer.evaluateResponse(saslToken);
23282312
return buildSaslResponse(
23292313
saslServer.isComplete() ? SaslState.SUCCESS : SaslState.CHALLENGE,
@@ -2338,9 +2322,8 @@ private void switchToSimple() {
23382322

23392323
private RpcSaslProto buildSaslResponse(SaslState state, byte[] replyToken) {
23402324
if (LOG.isDebugEnabled()) {
2341-
LOG.debug("Will send " + state + " token of size "
2342-
+ ((replyToken != null) ? replyToken.length : null)
2343-
+ " from saslServer.");
2325+
LOG.debug("Will send {} token of size {} from saslServer.", state,
2326+
((replyToken != null) ? replyToken.length : null));
23442327
}
23452328
RpcSaslProto.Builder response = RpcSaslProto.newBuilder();
23462329
response.setState(state);
@@ -2664,10 +2647,8 @@ private void processConnectionContext(RpcWritable.Buffer buffer)
26642647
*/
26652648
private void unwrapPacketAndProcessRpcs(byte[] inBuf)
26662649
throws IOException, InterruptedException {
2667-
if (LOG.isDebugEnabled()) {
2668-
LOG.debug("Have read input token of size " + inBuf.length
2669-
+ " for processing by saslServer.unwrap()");
2670-
}
2650+
LOG.debug("Have read input token of size {} for processing by saslServer.unwrap()",
2651+
inBuf.length);
26712652
inBuf = saslServer.unwrap(inBuf, 0, inBuf.length);
26722653
ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(
26732654
inBuf));
@@ -2729,9 +2710,7 @@ private void processOneRpc(ByteBuffer bb)
27292710
getMessage(RpcRequestHeaderProto.getDefaultInstance(), buffer);
27302711
callId = header.getCallId();
27312712
retry = header.getRetryCount();
2732-
if (LOG.isDebugEnabled()) {
2733-
LOG.debug(" got #" + callId);
2734-
}
2713+
LOG.debug(" got #{}", callId);
27352714
checkRpcHeaders(header);
27362715

27372716
if (callId < 0) { // callIds typically used during connection setup
@@ -2746,11 +2725,8 @@ private void processOneRpc(ByteBuffer bb)
27462725
} catch (RpcServerException rse) {
27472726
// inform client of error, but do not rethrow else non-fatal
27482727
// exceptions will close connection!
2749-
if (LOG.isDebugEnabled()) {
2750-
LOG.debug(Thread.currentThread().getName() +
2751-
": processOneRpc from client " + this +
2752-
" threw exception [" + rse + "]");
2753-
}
2728+
LOG.debug("{}: processOneRpc from client {} threw exception [{}]",
2729+
Thread.currentThread().getName(), this, rse);
27542730
// use the wrapped exception if there is one.
27552731
Throwable t = (rse.getCause() != null) ? rse.getCause() : rse;
27562732
final RpcCall call = new RpcCall(this, callId, retry);
@@ -2962,9 +2938,7 @@ private void authorizeConnection() throws RpcServerException {
29622938
ProxyUsers.authorize(user, this.getHostAddress());
29632939
}
29642940
authorize(user, protocolName, getHostInetAddress());
2965-
if (LOG.isDebugEnabled()) {
2966-
LOG.debug("Successfully authorized " + connectionContext);
2967-
}
2941+
LOG.debug("Successfully authorized {}.", connectionContext);
29682942
rpcMetrics.incrAuthorizationSuccesses();
29692943
} catch (AuthorizationException ae) {
29702944
LOG.info("Connection from " + this
@@ -3081,7 +3055,7 @@ public Handler(int instanceNumber) {
30813055

30823056
@Override
30833057
public void run() {
3084-
LOG.debug(Thread.currentThread().getName() + ": starting");
3058+
LOG.debug("{}: starting", Thread.currentThread().getName());
30853059
SERVER.set(Server.this);
30863060
while (running) {
30873061
TraceScope traceScope = null;
@@ -3115,9 +3089,7 @@ public void run() {
31153089
call = null;
31163090
continue;
31173091
}
3118-
if (LOG.isDebugEnabled()) {
3119-
LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
3120-
}
3092+
LOG.debug("{}: {} for RpcKind {}.", Thread.currentThread().getName(), call, call.rpcKind);
31213093
CurCall.set(call);
31223094
if (call.span != null) {
31233095
traceScope = tracer.activateSpan(call.span);
@@ -3152,15 +3124,14 @@ public void run() {
31523124
IOUtils.cleanupWithLogger(LOG, traceScope);
31533125
if (call != null) {
31543126
updateMetrics(call, startTimeNanos, connDropped);
3155-
ProcessingDetails.LOG.debug(
3156-
"Served: [{}]{} name={} user={} details={}",
3127+
ProcessingDetails.LOG.debug("Served: [{}]{} name={} user={} details={}",
31573128
call, (call.isResponseDeferred() ? ", deferred" : ""),
31583129
call.getDetailedMetricsName(), call.getRemoteUser(),
31593130
call.getProcessingDetails());
31603131
}
31613132
}
31623133
}
3163-
LOG.debug(Thread.currentThread().getName() + ": exiting");
3134+
LOG.debug("{}: exiting", Thread.currentThread().getName());
31643135
}
31653136

31663137
private void requeueCall(Call call)
@@ -3389,14 +3360,13 @@ private List<AuthMethod> getAuthMethods(SecretManager<?> secretManager,
33893360
" authentication requires a secret manager");
33903361
}
33913362
} else if (secretManager != null) {
3392-
LOG.debug(AuthenticationMethod.TOKEN +
3393-
" authentication enabled for secret manager");
3363+
LOG.debug("{} authentication enabled for secret manager", AuthenticationMethod.TOKEN);
33943364
// most preferred, go to the front of the line!
33953365
authMethods.add(AuthenticationMethod.TOKEN.getAuthMethod());
33963366
}
33973367
authMethods.add(confAuthenticationMethod.getAuthMethod());
33983368

3399-
LOG.debug("Server accepts auth methods:" + authMethods);
3369+
LOG.debug("Server accepts auth methods:{}", authMethods);
34003370
return authMethods;
34013371
}
34023372

@@ -3556,9 +3526,7 @@ private void wrapWithSasl(RpcCall call) throws IOException {
35563526
synchronized (call.connection.saslServer) {
35573527
token = call.connection.saslServer.wrap(token, 0, token.length);
35583528
}
3559-
if (LOG.isDebugEnabled())
3560-
LOG.debug("Adding saslServer wrapped token of size " + token.length
3561-
+ " as call response.");
3529+
LOG.debug("Adding saslServer wrapped token of size {} as call response.", token.length);
35623530
// rebuild with sasl header and payload
35633531
RpcResponseHeaderProto saslHeader = RpcResponseHeaderProto.newBuilder()
35643532
.setCallId(AuthProtocol.SASL.callId)
@@ -4004,21 +3972,17 @@ Connection register(SocketChannel channel, int ingressPort,
40043972
Connection connection = new Connection(channel, Time.now(),
40053973
ingressPort, isOnAuxiliaryPort);
40063974
add(connection);
4007-
if (LOG.isDebugEnabled()) {
4008-
LOG.debug("Server connection from " + connection +
4009-
"; # active connections: " + size() +
4010-
"; # queued calls: " + callQueue.size());
4011-
}
3975+
LOG.debug("Server connection from {}; # active connections: {}; # queued calls: {}.",
3976+
connection, size(), callQueue.size());
40123977
return connection;
40133978
}
40143979

40153980
boolean close(Connection connection) {
40163981
boolean exists = remove(connection);
40173982
if (exists) {
40183983
if (LOG.isDebugEnabled()) {
4019-
LOG.debug(Thread.currentThread().getName() +
4020-
": disconnecting client " + connection +
4021-
". Number of active connections: "+ size());
3984+
LOG.debug("{}: disconnecting client {}. Number of active connections: {}.",
3985+
Thread.currentThread().getName(), connection, size());
40223986
}
40233987
// only close if actually removed to avoid double-closing due
40243988
// to possible races
@@ -4080,9 +4044,7 @@ public void run() {
40804044
if (!running) {
40814045
return;
40824046
}
4083-
if (LOG.isDebugEnabled()) {
4084-
LOG.debug(Thread.currentThread().getName()+": task running");
4085-
}
4047+
LOG.debug("{}: task running", Thread.currentThread().getName());
40864048
try {
40874049
closeIdle(false);
40884050
} finally {

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -522,8 +522,7 @@ protected Node chooseRandom(final String scope, String excludedScope,
522522
}
523523
}
524524
if (numOfDatanodes <= 0) {
525-
LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\")."
526-
+ " numOfDatanodes={}",
525+
LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\"). numOfDatanodes={}",
527526
scope, excludedScope, numOfDatanodes);
528527
return null;
529528
}
@@ -539,10 +538,12 @@ protected Node chooseRandom(final String scope, String excludedScope,
539538
netlock.readLock().unlock();
540539
}
541540
}
542-
LOG.debug("Choosing random from {} available nodes on node {},"
543-
+ " scope={}, excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
544-
availableNodes, innerNode, scope, excludedScope, excludedNodes,
545-
numOfDatanodes);
541+
if (LOG.isDebugEnabled()) {
542+
LOG.debug("Choosing random from {} available nodes on node {}, scope={},"
543+
+ " excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
544+
availableNodes, innerNode, scope, excludedScope, excludedNodes,
545+
numOfDatanodes);
546+
}
546547
Node ret = null;
547548
if (availableNodes > 0) {
548549
ret = chooseRandom(innerNode, node, excludedNodes, numOfDatanodes,

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

Lines changed: 5 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -479,10 +479,9 @@ public void recoverUnfinalizedSegments() throws IOException {
479479
LOG.info("Successfully started new epoch " + loggers.getEpoch());
480480

481481
if (LOG.isDebugEnabled()) {
482-
LOG.debug("newEpoch(" + loggers.getEpoch() + ") responses:\n" +
483-
QuorumCall.mapToString(resps));
482+
LOG.debug("newEpoch({}) responses:\n{}", loggers.getEpoch(), QuorumCall.mapToString(resps));
484483
}
485-
484+
486485
long mostRecentSegmentTxId = Long.MIN_VALUE;
487486
for (NewEpochResponseProto r : resps.values()) {
488487
if (r.hasLastSegmentTxId()) {
@@ -518,10 +517,7 @@ public void selectInputStreams(Collection<EditLogInputStream> streams,
518517
// the cache used for RPC calls is not enabled; fall back to using the
519518
// streaming mechanism to serve such requests
520519
if (inProgressOk && inProgressTailingEnabled) {
521-
if (LOG.isDebugEnabled()) {
522-
LOG.debug("Tailing edits starting from txn ID " + fromTxnId +
523-
" via RPC mechanism");
524-
}
520+
LOG.debug("Tailing edits starting from txn ID {} via RPC mechanism", fromTxnId);
525521
try {
526522
Collection<EditLogInputStream> rpcStreams = new ArrayList<>();
527523
selectRpcInputStreams(rpcStreams, fromTxnId, onlyDurableTxns);
@@ -585,8 +581,8 @@ private void selectRpcInputStreams(Collection<EditLogInputStream> streams,
585581
int maxAllowedTxns = !onlyDurableTxns ? highestTxnCount :
586582
responseCounts.get(responseCounts.size() - loggers.getMajoritySize());
587583
if (maxAllowedTxns == 0) {
588-
LOG.debug("No new edits available in logs; requested starting from " +
589-
"ID {}", fromTxnId);
584+
LOG.debug("No new edits available in logs; requested starting from ID {}",
585+
fromTxnId);
590586
return;
591587
}
592588
LogAction logAction = selectInputStreamLogHelper.record(fromTxnId);

0 commit comments

Comments
 (0)