1
0
فهرست منبع

HDDS-2260. Avoid evaluation of LOG.trace and LOG.debug statement in the read/write path (HDDS). (#1612)

Siddharth 5 سال پیش
والد
کامیت
15a9beed1b
55فایلهای تغییر یافته به همراه420 افزوده شده و 222 حذف شده
  1. 11 6
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
  2. 19 9
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
  3. 7 3
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java
  4. 4 2
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
  5. 24 16
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
  6. 6 2
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/CommitWatcher.java
  7. 4 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
  8. 3 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/HddsConfServlet.java
  9. 4 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
  10. 14 7
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java
  11. 8 4
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
  12. 3 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/token/BlockTokenVerifier.java
  13. 6 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/token/OzoneBlockTokenSelector.java
  14. 3 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultProfile.java
  15. 4 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java
  16. 3 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/tracing/StringCodec.java
  17. 6 3
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java
  18. 4 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/HddsVersionInfo.java
  19. 13 7
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/LevelDBStore.java
  20. 10 6
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java
  21. 1 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseCallbackExecutor.java
  22. 3 3
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
  23. 6 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
  24. 8 3
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
  25. 6 5
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
  26. 7 6
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
  27. 3 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
  28. 4 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
  29. 18 10
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
  30. 25 15
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
  31. 9 5
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolumeChecker.java
  32. 5 3
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/ThrottledAsyncChecker.java
  33. 4 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueBlockIterator.java
  34. 56 28
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
  35. 4 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
  36. 5 3
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
  37. 5 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
  38. 4 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
  39. 3 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScanner.java
  40. 1 1
      hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
  41. 7 3
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
  42. 7 5
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
  43. 8 4
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/command/CommandStatusReportHandler.java
  44. 4 2
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
  45. 4 2
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerActionsHandler.java
  46. 3 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
  47. 4 2
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java
  48. 4 2
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackAware.java
  49. 16 6
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerAttribute.java
  50. 4 2
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
  51. 7 3
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
  52. 3 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
  53. 3 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
  54. 3 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java
  55. 10 8
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java

+ 11 - 6
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java

@@ -158,8 +158,10 @@ public class XceiverClientGrpc extends XceiverClientSpi {
 
     // Add credential context to the client call
     String userName = UserGroupInformation.getCurrentUser().getShortUserName();
-    LOG.debug("Nodes in pipeline : {}", pipeline.getNodes().toString());
-    LOG.debug("Connecting to server : {}", dn.getIpAddress());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Nodes in pipeline : {}", pipeline.getNodes().toString());
+      LOG.debug("Connecting to server : {}", dn.getIpAddress());
+    }
     NettyChannelBuilder channelBuilder =
         NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext()
             .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
@@ -283,7 +285,9 @@ public class XceiverClientGrpc extends XceiverClientSpi {
     }
     for (DatanodeDetails dn : datanodeList) {
       try {
-        LOG.debug("Executing command " + request + " on datanode " + dn);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Executing command " + request + " on datanode " + dn);
+        }
         // In case the command gets retried on a 2nd datanode,
         // sendCommandAsyncCall will create a new channel and async stub
         // in case these don't exist for the specific datanode.
@@ -377,9 +381,10 @@ public class XceiverClientGrpc extends XceiverClientSpi {
     if (!isConnected(channel)) {
       reconnect(dn, token);
     }
-
-    LOG.debug("Send command {} to datanode {}", request.getCmdType().toString(),
-        dn.getNetworkFullPath());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Send command {} to datanode {}",
+          request.getCmdType().toString(), dn.getNetworkFullPath());
+    }
     final CompletableFuture<ContainerCommandResponseProto> replyFuture =
         new CompletableFuture<>();
     semaphore.acquire();

+ 19 - 9
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java

@@ -170,8 +170,10 @@ public final class XceiverClientRatis extends XceiverClientSpi {
 
   @Override
   public void connect() throws Exception {
-    LOG.debug("Connecting to pipeline:{} datanode:{}", getPipeline().getId(),
-        RatisHelper.toRaftPeerId(pipeline.getFirstNode()));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to pipeline:{} datanode:{}", getPipeline().getId(),
+          RatisHelper.toRaftPeerId(pipeline.getFirstNode()));
+    }
     // TODO : XceiverClient ratis should pass the config value of
     // maxOutstandingRequests so as to set the upper bound on max no of async
     // requests to be handled by raft client
@@ -223,10 +225,14 @@ public final class XceiverClientRatis extends XceiverClientSpi {
           = ContainerCommandRequestMessage.toMessage(
               request, TracingUtil.exportCurrentSpan());
       if (HddsUtils.isReadOnly(request)) {
-        LOG.debug("sendCommandAsync ReadOnly {}", message);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("sendCommandAsync ReadOnly {}", message);
+        }
         return getClient().sendReadOnlyAsync(message);
       } else {
-        LOG.debug("sendCommandAsync {}", message);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("sendCommandAsync {}", message);
+        }
         return getClient().sendAsync(message);
       }
     }
@@ -258,7 +264,9 @@ public final class XceiverClientRatis extends XceiverClientSpi {
       clientReply.setLogIndex(commitIndex);
       return clientReply;
     }
-    LOG.debug("commit index : {} watch timeout : {}", index, timeout);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("commit index : {} watch timeout : {}", index, timeout);
+    }
     RaftClientReply reply;
     try {
       CompletableFuture<RaftClientReply> replyFuture = getClient()
@@ -310,10 +318,12 @@ public final class XceiverClientRatis extends XceiverClientSpi {
     metrics.incrPendingContainerOpsMetrics(request.getCmdType());
     CompletableFuture<ContainerCommandResponseProto> containerCommandResponse =
         raftClientReply.whenComplete((reply, e) -> {
-          LOG.debug("received reply {} for request: cmdType={} containerID={}"
-                  + " pipelineID={} traceID={} exception: {}", reply,
-              request.getCmdType(), request.getContainerID(),
-              request.getPipelineID(), request.getTraceID(), e);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("received reply {} for request: cmdType={} containerID={}"
+                    + " pipelineID={} traceID={} exception: {}", reply,
+                request.getCmdType(), request.getContainerID(),
+                request.getPipelineID(), request.getTraceID(), e);
+          }
           metrics.decrPendingContainerOpsMetrics(request.getCmdType());
           metrics.addContainerOpsLatency(request.getCmdType(),
               Time.monotonicNowNanos() - requestTime);

+ 7 - 3
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java

@@ -165,8 +165,10 @@ public class ContainerOperationClient implements ScmClient {
 
     // TODO : Should we change the state on the client side ??
     // That makes sense, but it is not needed for the client to work.
-    LOG.debug("Pipeline creation successful. Pipeline: {}",
-        pipeline.toString());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Pipeline creation successful. Pipeline: {}",
+          pipeline.toString());
+    }
   }
 
   @Override
@@ -379,7 +381,9 @@ public class ContainerOperationClient implements ScmClient {
       throws IOException {
     XceiverClientSpi client = null;
     try {
-      LOG.debug("Close container {}", pipeline);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Close container {}", pipeline);
+      }
       /*
       TODO: two orders here, revisit this later:
       1. close on SCM first, then on data node

+ 4 - 2
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java

@@ -155,8 +155,10 @@ public class BlockInputStream extends InputStream implements Seekable {
     boolean success = false;
     List<ChunkInfo> chunks;
     try {
-      LOG.debug("Initializing BlockInputStream for get key to access {}",
-          blockID.getContainerID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Initializing BlockInputStream for get key to access {}",
+            blockID.getContainerID());
+      }
 
       if (token != null) {
         UserGroupInformation.getCurrentUser().addToken(token);

+ 24 - 16
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java

@@ -392,22 +392,26 @@ public class BlockOutputStream extends OutputStream {
               .equals(responseBlockID.getContainerBlockID()));
           // updates the bcsId of the block
           blockID = responseBlockID;
-          LOG.debug(
-              "Adding index " + asyncReply.getLogIndex() + " commitMap size "
-                  + commitWatcher.getCommitInfoMapSize() + " flushLength "
-                  + flushPos + " numBuffers " + byteBufferList.size()
-                  + " blockID " + blockID + " bufferPool size" + bufferPool
-                  .getSize() + " currentBufferIndex " + bufferPool
-                  .getCurrentBufferIndex());
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
           // for standalone protocol, logIndex will always be 0.
           commitWatcher
               .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
         }
         return e;
       }, responseExecutor).exceptionally(e -> {
-        LOG.debug(
-            "putBlock failed for blockID " + blockID + " with exception " + e
-                .getLocalizedMessage());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "putBlock failed for blockID " + blockID + " with exception " + e
+                  .getLocalizedMessage());
+        }
         CompletionException ce =  new CompletionException(e);
         setIoException(ce);
         throw ce;
@@ -609,9 +613,11 @@ public class BlockOutputStream extends OutputStream {
         }
         return e;
       }, responseExecutor).exceptionally(e -> {
-        LOG.debug(
-            "writing chunk failed " + chunkInfo.getChunkName() + " blockID "
-                + blockID + " with exception " + e.getLocalizedMessage());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "writing chunk failed " + chunkInfo.getChunkName() + " blockID "
+                  + blockID + " with exception " + e.getLocalizedMessage());
+        }
         CompletionException ce = new CompletionException(e);
         setIoException(ce);
         throw ce;
@@ -620,9 +626,11 @@ public class BlockOutputStream extends OutputStream {
       throw new IOException(
           "Unexpected Storage Container Exception: " + e.toString(), e);
     }
-    LOG.debug(
-        "writing chunk " + chunkInfo.getChunkName() + " blockID " + blockID
-            + " length " + effectiveChunkSize);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "writing chunk " + chunkInfo.getChunkName() + " blockID " + blockID
+              + " length " + effectiveChunkSize);
+    }
     containerBlockData.addChunks(chunkInfo);
   }
 

+ 6 - 2
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/CommitWatcher.java

@@ -131,7 +131,9 @@ public class CommitWatcher {
       long index =
           commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v).min()
               .getAsLong();
-      LOG.debug("waiting for first index " + index + " to catch up");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("waiting for first index " + index + " to catch up");
+      }
       return watchForCommit(index);
     } else {
       return null;
@@ -153,7 +155,9 @@ public class CommitWatcher {
       long index =
           commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v).max()
               .getAsLong();
-      LOG.debug("waiting for last flush Index " + index + " to catch up");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("waiting for last flush Index " + index + " to catch up");
+      }
       return watchForCommit(index);
     } else {
       return null;

+ 4 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java

@@ -423,8 +423,10 @@ public final class HddsUtils {
         InvocationTargetException e) {
 
       // Fallback
-      LOG.trace("Registering MBean {} without additional properties {}",
-          mBeanName, jmxProperties);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Registering MBean {} without additional properties {}",
+            mBeanName, jmxProperties);
+      }
       return MBeans.register(serviceName, mBeanName, mBean);
     }
   }

+ 3 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/HddsConfServlet.java

@@ -171,7 +171,9 @@ public class HddsConfServlet extends HttpServlet {
           Properties properties = config.getAllPropertiesByTag(tag);
           propMap.put(tag, properties);
         } else {
-          LOG.debug("Not a valid tag" + tag);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Not a valid tag" + tag);
+          }
         }
       }
       out.write(gson.toJsonTree(propMap).toString());

+ 4 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java

@@ -183,7 +183,10 @@ public interface RatisHelper {
   static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
       RaftGroup group, RetryPolicy retryPolicy, int maxOutStandingRequest,
       GrpcTlsConfig tlsConfig, TimeDuration clientRequestTimeout) {
-    LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("newRaftClient: {}, leader={}, group={}",
+          rpcType, leader, group);
+    }
     final RaftProperties properties = new RaftProperties();
     RaftConfigKeys.Rpc.setType(properties, rpcType);
     RaftClientConfigKeys.Rpc

+ 14 - 7
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java

@@ -109,7 +109,9 @@ public class NetworkTopologyImpl implements NetworkTopology{
 
     if (add) {
       LOG.info("Added a new node: " + node.getNetworkFullPath());
-      LOG.debug("NetworkTopology became:\n{}", this);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("NetworkTopology became:\n{}", this);
+      }
     }
   }
 
@@ -131,7 +133,9 @@ public class NetworkTopologyImpl implements NetworkTopology{
       netlock.writeLock().unlock();
     }
     LOG.info("Removed a node: " + node.getNetworkFullPath());
-    LOG.debug("NetworkTopology became:\n{}", this);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NetworkTopology became:\n{}", this);
+    }
   }
 
   /**
@@ -558,11 +562,14 @@ public class NetworkTopologyImpl implements NetworkTopology{
       ret = ((InnerNode)scopeNode).getLeaf(nodeIndex, mutableExcludedScopes,
           mutableExNodes, ancestorGen);
     }
-    LOG.debug("Choosing node[index={},random={}] from \"{}\" available nodes" +
-            " scope=\"{}\", excludedScope=\"{}\", excludeNodes=\"{}\".",
-        nodeIndex, (leafIndex == -1 ? "true" : "false"), availableNodes,
-        scopeNode.getNetworkFullPath(), excludedScopes, excludedNodes);
-    LOG.debug("Chosen node = {}", (ret == null ? "not found" : ret.toString()));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Choosing node[index={},random={}] from \"{}\" available " +
+              "nodes, scope=\"{}\", excludedScope=\"{}\", excludeNodes=\"{}\".",
+          nodeIndex, (leafIndex == -1 ? "true" : "false"), availableNodes,
+          scopeNode.getNetworkFullPath(), excludedScopes, excludedNodes);
+      LOG.debug("Chosen node = {}", (ret == null ? "not found" :
+          ret.toString()));
+    }
     return ret;
   }
 

+ 8 - 4
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java

@@ -192,8 +192,10 @@ public final class Pipeline {
           }
         }
       }
-      LOG.debug("Serialize pipeline {} with nodesInOrder{ }", id.toString(),
-          nodes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Serialize pipeline {} with nodesInOrder{ }", id.toString(),
+            nodes);
+      }
     }
     return builder.build();
   }
@@ -339,8 +341,10 @@ public final class Pipeline {
             nodeIndex--;
           }
         }
-        LOG.debug("Deserialize nodesInOrder {} in pipeline {}", nodesWithOrder,
-            id.toString());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Deserialize nodesInOrder {} in pipeline {}",
+              nodesWithOrder, id.toString());
+        }
         pipeline.setNodesInOrder(nodesWithOrder);
       } else if (nodesInOrder != null){
         // This branch is for pipeline clone

+ 3 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/token/BlockTokenVerifier.java

@@ -68,7 +68,9 @@ public class BlockTokenVerifier implements TokenVerifier {
       OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier();
       try {
         token.decodeFromUrlString(tokenStr);
-        LOGGER.debug("Verifying token:{} for user:{} ", token, user);
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("Verifying token:{} for user:{} ", token, user);
+        }
         ByteArrayInputStream buf = new ByteArrayInputStream(
             token.getIdentifier());
         DataInputStream in = new DataInputStream(buf);

+ 6 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/token/OzoneBlockTokenSelector.java

@@ -47,7 +47,9 @@ public class OzoneBlockTokenSelector implements
     for (Token<? extends TokenIdentifier> token : tokens) {
       if (OzoneBlockTokenIdentifier.KIND_NAME.equals(token.getKind())
           && token.getService().equals(service)) {
-        LOG.trace("Getting token for service:{}", service);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Getting token for service:{}", service);
+        }
         return (Token<OzoneBlockTokenIdentifier>) token;
       }
     }
@@ -66,7 +68,9 @@ public class OzoneBlockTokenSelector implements
     for (Token<? extends TokenIdentifier> token : tokens) {
       if (OzoneBlockTokenIdentifier.KIND_NAME.equals(token.getKind())
           && token.getService().equals(service)) {
-        LOG.trace("Getting token for service:{}", service);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Getting token for service:{}", service);
+        }
         return (Token<OzoneBlockTokenIdentifier>) token;
       }
     }

+ 3 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultProfile.java

@@ -236,7 +236,9 @@ public class DefaultProfile implements PKIProfile {
       try {
         final InetAddress byAddress = InetAddress.getByAddress(
             Hex.decodeHex(value.substring(1)));
-        LOG.debug("Host Name/IP Address : {}", byAddress.toString());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Host Name/IP Address : {}", byAddress.toString());
+        }
         return true;
       } catch (UnknownHostException | DecoderException e) {
         return false;

+ 4 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java

@@ -108,8 +108,10 @@ public class HDDSKeyGenerator {
    */
   public KeyPair generateKey(int size, String algorithm, String provider)
       throws NoSuchProviderException, NoSuchAlgorithmException {
-    LOG.debug("Generating key pair using size:{}, Algorithm:{}, Provider:{}",
-        size, algorithm, provider);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Generating key pair using size:{}, Algorithm:{}, Provider:{}",
+          size, algorithm, provider);
+    }
     KeyPairGenerator generator = KeyPairGenerator
         .getInstance(algorithm, provider);
     generator.initialize(size);

+ 3 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/tracing/StringCodec.java

@@ -45,7 +45,9 @@ public class StringCodec implements Codec<StringBuilder> {
     if (value != null && !value.equals("")) {
       String[] parts = value.split(":");
       if (parts.length != 4) {
-        LOG.debug("MalformedTracerStateString: {}", value);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("MalformedTracerStateString: {}", value);
+        }
         throw new MalformedTracerStateStringException(value);
       } else {
         String traceId = parts[0];

+ 6 - 3
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java

@@ -102,15 +102,18 @@ public abstract class BackgroundService {
   public class PeriodicalTask implements Runnable {
     @Override
     public synchronized void run() {
-      LOG.debug("Running background service : {}", serviceName);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Running background service : {}", serviceName);
+      }
       BackgroundTaskQueue tasks = getTasks();
       if (tasks.isEmpty()) {
         // No task found, or some problems to init tasks
         // return and retry in next interval.
         return;
       }
-
-      LOG.debug("Number of background tasks to execute : {}", tasks.size());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Number of background tasks to execute : {}", tasks.size());
+      }
       CompletionService<BackgroundTaskResult> taskCompletionService =
           new ExecutorCompletionService<>(exec);
 

+ 4 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/HddsVersionInfo.java

@@ -50,7 +50,9 @@ public final class HddsVersionInfo {
         "Compiled with protoc " + HDDS_VERSION_INFO.getProtocVersion());
     System.out.println(
         "From source with checksum " + HDDS_VERSION_INFO.getSrcChecksum());
-    LOG.debug("This command was run using " +
-        ClassUtil.findContainingJar(HddsVersionInfo.class));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("This command was run using " +
+          ClassUtil.findContainingJar(HddsVersionInfo.class));
+    }
   }
 }

+ 13 - 7
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/LevelDBStore.java

@@ -77,7 +77,9 @@ public class LevelDBStore implements MetadataStore {
 
   private void openDB(File dbPath, Options options) throws IOException {
     if (dbPath.getParentFile().mkdirs()) {
-      LOG.debug("Db path {} created.", dbPath.getParentFile());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Db path {} created.", dbPath.getParentFile());
+      }
     }
     db = JniDBFactory.factory.open(dbPath, options);
     if (LOG.isDebugEnabled()) {
@@ -370,17 +372,21 @@ public class LevelDBStore implements MetadataStore {
             int scanned = filter.getKeysScannedNum();
             int hinted = filter.getKeysHintedNum();
             if (scanned > 0 || hinted > 0) {
-              LOG.debug(
-                  "getRangeKVs ({}) numOfKeysScanned={}, numOfKeysHinted={}",
-                  filter.getClass().getSimpleName(), filter.getKeysScannedNum(),
-                  filter.getKeysHintedNum());
+              if (LOG.isDebugEnabled()) {
+                LOG.debug(
+                    "getRangeKVs ({}) numOfKeysScanned={}, numOfKeysHinted={}",
+                    filter.getClass().getSimpleName(),
+                    filter.getKeysScannedNum(), filter.getKeysHintedNum());
+              }
             }
           }
         }
         long end = System.currentTimeMillis();
         long timeConsumed = end - start;
-        LOG.debug("Time consumed for getRangeKVs() is {}ms,"
-            + " result length is {}.", timeConsumed, result.size());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Time consumed for getRangeKVs() is {}ms,"
+              + " result length is {}.", timeConsumed, result.size());
+        }
       }
     }
     return result;

+ 10 - 6
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java

@@ -79,9 +79,11 @@ public final class ContainerCommandRequestPBHelper {
         auditParams.put("blockData",
             BlockData.getFromProtoBuf(msg.getPutBlock().getBlockData())
                 .toString());
-      }catch (IOException ex){
-        LOG.trace("Encountered error parsing BlockData from protobuf:"
-            + ex.getMessage());
+      } catch (IOException ex){
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Encountered error parsing BlockData from protobuf: "
+              + ex.getMessage());
+        }
         return null;
       }
       return auditParams;
@@ -134,9 +136,11 @@ public final class ContainerCommandRequestPBHelper {
         auditParams.put("blockData",
             BlockData.getFromProtoBuf(msg.getPutSmallFile()
                 .getBlock().getBlockData()).toString());
-      }catch (IOException ex){
-        LOG.trace("Encountered error parsing BlockData from protobuf:"
-            + ex.getMessage());
+      } catch (IOException ex){
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Encountered error parsing BlockData from protobuf: "
+              + ex.getMessage());
+        }
       }
       return auditParams;
 

+ 1 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseCallbackExecutor.java

@@ -49,7 +49,7 @@ public class LeaseCallbackExecutor<T> implements Runnable {
 
   @Override
   public void run() {
-    if(LOG.isDebugEnabled()) {
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Executing callbacks for lease on {}", resource);
     }
     for(Callable<Void> callback : callbacks) {

+ 3 - 3
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java

@@ -110,7 +110,7 @@ public class LeaseManager<T> {
   public synchronized Lease<T> acquire(T resource, long timeout)
       throws LeaseAlreadyExistException {
     checkStatus();
-    if(LOG.isDebugEnabled()) {
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Acquiring lease on {} for {} milliseconds", resource, timeout);
     }
     if(activeLeases.containsKey(resource)) {
@@ -150,7 +150,7 @@ public class LeaseManager<T> {
   public synchronized void release(T resource)
       throws LeaseNotFoundException {
     checkStatus();
-    if(LOG.isDebugEnabled()) {
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Releasing lease on {}", resource);
     }
     Lease<T> lease = activeLeases.remove(resource);
@@ -206,7 +206,7 @@ public class LeaseManager<T> {
 
     @Override
     public void run() {
-      while(monitor) {
+      while (monitor) {
         LOG.debug("{}-LeaseMonitor: checking for lease expiry", name);
         long sleepTime = Long.MAX_VALUE;
 

+ 6 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java

@@ -65,9 +65,11 @@ public class ContainerSet {
     Preconditions.checkNotNull(container, "container cannot be null");
 
     long containerId = container.getContainerData().getContainerID();
-    if(containerMap.putIfAbsent(containerId, container) == null) {
-      LOG.debug("Container with container Id {} is added to containerMap",
-          containerId);
+    if (containerMap.putIfAbsent(containerId, container) == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Container with container Id {} is added to containerMap",
+            containerId);
+      }
       // wish we could have done this from ContainerData.setState
       container.getContainerData().commitSpace();
       return true;
@@ -100,7 +102,7 @@ public class ContainerSet {
     Preconditions.checkState(containerId >= 0,
         "Container Id cannot be negative.");
     Container<?> removed = containerMap.remove(containerId);
-    if(removed == null) {
+    if (removed == null) {
       LOG.debug("Container with containerId {} is not present in " +
           "containerMap", containerId);
       return false;

+ 8 - 3
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java

@@ -155,8 +155,10 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
   private ContainerCommandResponseProto dispatchRequest(
       ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
     Preconditions.checkNotNull(msg);
-    LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
-        msg.getTraceID());
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
+          msg.getTraceID());
+    }
 
     AuditAction action = ContainerCommandRequestPBHelper.getAuditAction(
         msg.getCmdType());
@@ -553,7 +555,10 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
       }
       break;
 
-    default: LOG.debug("Invalid audit event status - " + result);
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Invalid audit event status - " + result);
+      }
     }
   }
 

+ 6 - 5
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java

@@ -55,11 +55,12 @@ public class RandomContainerDeletionChoosingPolicy
       if (currentCount < count) {
         result.add(entry);
         currentCount++;
-
-        LOG.debug("Select container {} for block deletion, "
-            + "pending deletion blocks num: {}.",
-            entry.getContainerID(),
-            ((KeyValueContainerData)entry).getNumPendingDeletionBlocks());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Select container {} for block deletion, "
+                  + "pending deletion blocks num: {}.",
+              entry.getContainerID(),
+              ((KeyValueContainerData) entry).getNumPendingDeletionBlocks());
+        }
       } else {
         break;
       }

+ 7 - 6
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java

@@ -69,12 +69,13 @@ public class TopNOrderedContainerDeletionChoosingPolicy
         if (entry.getNumPendingDeletionBlocks() > 0) {
           result.add(entry);
           currentCount++;
-
-          LOG.debug(
-              "Select container {} for block deletion, "
-                  + "pending deletion blocks num: {}.",
-              entry.getContainerID(),
-              entry.getNumPendingDeletionBlocks());
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Select container {} for block deletion, "
+                    + "pending deletion blocks num: {}.",
+                entry.getContainerID(),
+                entry.getNumPendingDeletionBlocks());
+          }
         } else {
           LOG.debug("Stop looking for next container, there is no"
               + " pending deletion block contained in remaining containers.");

+ 3 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java

@@ -207,7 +207,9 @@ public class EndpointStateMachine
           TimeUnit.MILLISECONDS.toSeconds(
               this.getMissedCount() * getScmHeartbeatInterval(this.conf)), ex);
     }
-    LOG.trace("Incrementing the Missed count. Ex : {}", ex);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Incrementing the Missed count. Ex : {}", ex);
+    }
     this.incMissed();
   }
 

+ 4 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java

@@ -114,8 +114,10 @@ public class CloseContainerCommandHandler implements CommandHandler {
         break;
       case UNHEALTHY:
       case INVALID:
-        LOG.debug("Cannot close the container #{}, the container is"
-            + " in {} state.", containerId, container.getContainerState());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Cannot close the container #{}, the container is"
+              + " in {} state.", containerId, container.getContainerState());
+        }
       default:
         break;
       }

+ 18 - 10
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java

@@ -196,9 +196,11 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
     }
 
     if (delTX.getTxID() < containerData.getDeleteTransactionId()) {
-      LOG.debug(String.format("Ignoring delete blocks for containerId: %d."
-              + " Outdated delete transactionId %d < %d", containerId,
-          delTX.getTxID(), containerData.getDeleteTransactionId()));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Ignoring delete blocks for containerId: %d."
+                + " Outdated delete transactionId %d < %d", containerId,
+            delTX.getTxID(), containerData.getDeleteTransactionId()));
+      }
       return;
     }
 
@@ -216,9 +218,11 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
               DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk);
           if (containerDB.getStore().get(deletingKeyBytes) != null
               || containerDB.getStore().get(deletedKeyBytes) != null) {
-            LOG.debug(String.format(
-                "Ignoring delete for block %d in container %d."
-                    + " Entry already added.", blk, containerId));
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(String.format(
+                  "Ignoring delete for block %d in container %d."
+                      + " Entry already added.", blk, containerId));
+            }
             continue;
           }
           // Found the block in container db,
@@ -228,8 +232,10 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
           try {
             containerDB.getStore().writeBatch(batch);
             newDeletionBlocks++;
-            LOG.debug("Transited Block {} to DELETING state in container {}",
-                blk, containerId);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Transited Block {} to DELETING state in container {}",
+                  blk, containerId);
+            }
           } catch (IOException e) {
             // if some blocks failed to delete, we fail this TX,
             // without sending this ACK to SCM, SCM will resend the TX
@@ -238,8 +244,10 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
                 "Failed to delete blocks for TXID = " + delTX.getTxID(), e);
           }
         } else {
-          LOG.debug("Block {} not found or already under deletion in"
-              + " container {}, skip deleting it.", blk, containerId);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Block {} not found or already under deletion in"
+                + " container {}, skip deleting it.", blk, containerId);
+          }
         }
       }
 

+ 25 - 15
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java

@@ -391,9 +391,11 @@ public class ContainerStateMachine extends BaseStateMachine {
 
   private ContainerCommandResponseProto dispatchCommand(
       ContainerCommandRequestProto requestProto, DispatcherContext context) {
-    LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", gid,
-        requestProto.getCmdType(), requestProto.getContainerID(),
-        requestProto.getPipelineID(), requestProto.getTraceID());
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", gid,
+          requestProto.getCmdType(), requestProto.getContainerID(),
+          requestProto.getPipelineID(), requestProto.getTraceID());
+    }
     if (isBlockTokenEnabled) {
       try {
         // ServerInterceptors intercepts incoming request and creates ugi.
@@ -409,7 +411,9 @@ public class ContainerStateMachine extends BaseStateMachine {
     }
     ContainerCommandResponseProto response =
         dispatcher.dispatch(requestProto, context);
-    LOG.trace("{}: response {}", gid, response);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("{}: response {}", gid, response);
+    }
     return response;
   }
 
@@ -462,9 +466,11 @@ public class ContainerStateMachine extends BaseStateMachine {
         }, chunkExecutor);
 
     writeChunkFutureMap.put(entryIndex, writeChunkFuture);
-    LOG.debug(gid + ": writeChunk writeStateMachineData : blockId " +
-        write.getBlockID() + " logIndex " + entryIndex + " chunkName "
-        + write.getChunkData().getChunkName());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(gid + ": writeChunk writeStateMachineData : blockId " +
+          write.getBlockID() + " logIndex " + entryIndex + " chunkName "
+          + write.getChunkData().getChunkName());
+    }
     // Remove the future once it finishes execution from the
     // writeChunkFutureMap.
     writeChunkFuture.thenApply(r -> {
@@ -480,10 +486,12 @@ public class ContainerStateMachine extends BaseStateMachine {
       } else {
         metrics.incNumBytesWrittenCount(
             requestProto.getWriteChunk().getChunkData().getLen());
-        LOG.debug(gid +
-            ": writeChunk writeStateMachineData  completed: blockId" +
-            write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
-            write.getChunkData().getChunkName());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(gid +
+              ": writeChunk writeStateMachineData  completed: blockId" +
+              write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
+              write.getChunkData().getChunkName());
+        }
         raftFuture.complete(r::toByteString);
         metrics.recordWriteStateMachineCompletion(
             Time.monotonicNowNanos() - startTime);
@@ -761,10 +769,12 @@ public class ContainerStateMachine extends BaseStateMachine {
           stateMachineHealthy.compareAndSet(true, false);
           ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole());
         } else {
-          LOG.debug(
-              "gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : "
-                  + "{} Container Result: {}", gid, r.getCmdType(), index,
-              r.getMessage(), r.getResult());
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : "
+                    + "{} Container Result: {}", gid, r.getCmdType(), index,
+                r.getMessage(), r.getResult());
+          }
           applyTransactionFuture.complete(r::toByteString);
           if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
             metrics.incNumBytesCommittedCount(

+ 9 - 5
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolumeChecker.java

@@ -181,10 +181,12 @@ public class HddsVolumeChecker {
     final long gap = timer.monotonicNow() - lastAllVolumesCheck;
     if (gap < minDiskCheckGapMs) {
       numSkippedChecks.incrementAndGet();
-      LOG.trace(
-          "Skipped checking all volumes, time since last check {} is less " +
-              "than the minimum gap between checks ({} ms).",
-          gap, minDiskCheckGapMs);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(
+            "Skipped checking all volumes, time since last check {} is less " +
+                "than the minimum gap between checks ({} ms).",
+            gap, minDiskCheckGapMs);
+      }
       return Collections.emptySet();
     }
 
@@ -314,7 +316,9 @@ public class HddsVolumeChecker {
       switch (result) {
       case HEALTHY:
       case DEGRADED:
-        LOG.debug("Volume {} is {}.", volume, result);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Volume {} is {}.", volume, result);
+        }
         markHealthy();
         break;
       case FAILED:

+ 5 - 3
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/ThrottledAsyncChecker.java

@@ -129,9 +129,11 @@ public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> {
           completedChecks.get(target);
       final long msSinceLastCheck = timer.monotonicNow() - result.completedAt;
       if (msSinceLastCheck < minMsBetweenChecks) {
-        LOG.debug("Skipped checking {}. Time since last check {}ms " +
-                "is less than the min gap {}ms.",
-            target, msSinceLastCheck, minMsBetweenChecks);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skipped checking {}. Time since last check {}ms " +
+                  "is less than the min gap {}ms.",
+              target, msSinceLastCheck, minMsBetweenChecks);
+        }
         return Optional.empty();
       }
     }

+ 4 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueBlockIterator.java

@@ -128,8 +128,10 @@ public class KeyValueBlockIterator implements BlockIterator<BlockData>,
       KeyValue block = blockIterator.next();
       if (blockFilter.filterKey(null, block.getKey(), null)) {
         nextBlock = BlockUtils.getBlockData(block.getValue());
-        LOG.trace("Block matching with filter found: blockID is : {} for " +
-            "containerID {}", nextBlock.getLocalID(), containerId);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Block matching with filter found: blockID is : {} for " +
+              "containerID {}", nextBlock.getLocalID(), containerId);
+        }
         return true;
       }
       hasNext();

+ 56 - 28
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java

@@ -206,8 +206,10 @@ public class KeyValueHandler extends Handler {
   ContainerCommandResponseProto handleCreateContainer(
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
     if (!request.hasCreateContainer()) {
-      LOG.debug("Malformed Create Container request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Create Container request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
     // Create Container request should be passed a null container as the
@@ -269,8 +271,10 @@ public class KeyValueHandler extends Handler {
   ContainerCommandResponseProto handleReadContainer(
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
     if (!request.hasReadContainer()) {
-      LOG.debug("Malformed Read Container request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Read Container request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -296,8 +300,10 @@ public class KeyValueHandler extends Handler {
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
     if (!request.hasUpdateContainer()) {
-      LOG.debug("Malformed Update Container request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Update Container request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -330,8 +336,10 @@ public class KeyValueHandler extends Handler {
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
     if (!request.hasDeleteContainer()) {
-      LOG.debug("Malformed Delete container request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Delete container request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -352,8 +360,10 @@ public class KeyValueHandler extends Handler {
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
     if (!request.hasCloseContainer()) {
-      LOG.debug("Malformed Update Container request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Update Container request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
     try {
@@ -379,8 +389,10 @@ public class KeyValueHandler extends Handler {
 
     long blockLength;
     if (!request.hasPutBlock()) {
-      LOG.debug("Malformed Put Key request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Put Key request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -415,8 +427,10 @@ public class KeyValueHandler extends Handler {
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
     if (!request.hasGetBlock()) {
-      LOG.debug("Malformed Get Key request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Get Key request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -454,8 +468,10 @@ public class KeyValueHandler extends Handler {
   ContainerCommandResponseProto handleGetCommittedBlockLength(
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
     if (!request.hasGetCommittedBlockLength()) {
-      LOG.debug("Malformed Get Key request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Get Key request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -490,8 +506,10 @@ public class KeyValueHandler extends Handler {
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
     if (!request.hasDeleteBlock()) {
-      LOG.debug("Malformed Delete Key request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Delete Key request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -521,8 +539,10 @@ public class KeyValueHandler extends Handler {
       DispatcherContext dispatcherContext) {
 
     if (!request.hasReadChunk()) {
-      LOG.debug("Malformed Read Chunk request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Read Chunk request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -589,8 +609,10 @@ public class KeyValueHandler extends Handler {
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
     if (!request.hasDeleteChunk()) {
-      LOG.debug("Malformed Delete Chunk request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Delete Chunk request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -632,8 +654,10 @@ public class KeyValueHandler extends Handler {
       DispatcherContext dispatcherContext) {
 
     if (!request.hasWriteChunk()) {
-      LOG.debug("Malformed Write Chunk request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Write Chunk request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 
@@ -687,8 +711,10 @@ public class KeyValueHandler extends Handler {
       DispatcherContext dispatcherContext) {
 
     if (!request.hasPutSmallFile()) {
-      LOG.debug("Malformed Put Small File request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Put Small File request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
     PutSmallFileRequestProto putSmallFileReq =
@@ -745,8 +771,10 @@ public class KeyValueHandler extends Handler {
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
 
     if (!request.hasGetSmallFile()) {
-      LOG.debug("Malformed Get Small File request. trace ID: {}",
-          request.getTraceID());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Malformed Get Small File request. trace ID: {}",
+            request.getTraceID());
+      }
       return ContainerUtils.malformedRequest(request);
     }
 

+ 4 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java

@@ -127,8 +127,10 @@ public final class ChunkUtils {
       return null;
     });
 
-    log.debug("Write Chunk completed for chunkFile: {}, size {}", chunkFile,
-        bufferSize);
+    if (log.isDebugEnabled()) {
+      log.debug("Write Chunk completed for chunkFile: {}, size {}", chunkFile,
+          bufferSize);
+    }
   }
 
   /**

+ 5 - 3
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java

@@ -120,9 +120,11 @@ public class BlockManagerImpl implements BlockManager {
       container.updateBlockCommitSequenceId(bcsId);
       // Increment keycount here
       container.getContainerData().incrKeyCount();
-      LOG.debug(
-          "Block " + data.getBlockID() + " successfully committed with bcsId "
-              + bcsId + " chunk size " + data.getChunks().size());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Block " + data.getBlockID() + " successfully committed with bcsId "
+                + bcsId + " chunk size " + data.getChunks().size());
+      }
       return data.getSize();
     }
   }

+ 5 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java

@@ -87,10 +87,11 @@ public class ChunkManagerImpl implements ChunkManager {
       boolean isOverwrite = ChunkUtils.validateChunkForOverwrite(
           chunkFile, info);
       File tmpChunkFile = getTmpChunkFile(chunkFile, dispatcherContext);
-
-      LOG.debug(
-          "writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file:{}",
-          info.getChunkName(), stage, chunkFile, tmpChunkFile);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file:{}",
+            info.getChunkName(), stage, chunkFile, tmpChunkFile);
+      }
 
       switch (stage) {
       case WRITE_DATA:

+ 4 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java

@@ -285,8 +285,10 @@ public class BlockDeletingService extends BackgroundService {
               File chunkFile = dataDir.toPath()
                   .resolve(chunkInfo.getChunkName()).toFile();
               if (FileUtils.deleteQuietly(chunkFile)) {
-                LOG.debug("block {} chunk {} deleted", blockName,
-                    chunkFile.getAbsolutePath());
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("block {} chunk {} deleted", blockName,
+                      chunkFile.getAbsolutePath());
+                }
               }
             }
             succeedBlocks.add(blockName);

+ 3 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScanner.java

@@ -69,7 +69,9 @@ public class ContainerDataScanner extends Thread {
 
   @Override
   public void run() {
-    LOG.trace("{}: thread starting.", this);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("{}: thread starting.", this);
+    }
     try {
       while (!stopping) {
         runIteration();

+ 1 - 1
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java

@@ -161,7 +161,7 @@ public class EventQueue implements EventPublisher, AutoCloseable {
 
         for (EventHandler handler : executorAndHandlers.getValue()) {
           queuedCount.incrementAndGet();
-          if (LOG.isTraceEnabled()) {
+          if (LOG.isDebugEnabled()) {
             LOG.debug(
                 "Delivering event {} to executor/handler {}: <json>{}</json>",
                 event.getName(),

+ 7 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java

@@ -153,7 +153,9 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
   public AllocatedBlock allocateBlock(final long size, ReplicationType type,
       ReplicationFactor factor, String owner, ExcludeList excludeList)
       throws IOException {
-    LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
+    }
     ScmUtils.preCheck(ScmOps.allocateBlock, safeModePrecheck);
     if (size < 0 || size > containerSize) {
       LOG.warn("Invalid block size requested : {}", size);
@@ -241,8 +243,10 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
       AllocatedBlock.Builder abb =  new AllocatedBlock.Builder()
           .setContainerBlockID(new ContainerBlockID(containerID, localID))
           .setPipeline(pipeline);
-      LOG.trace("New block allocated : {} Container ID: {}", localID,
-          containerID);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("New block allocated : {} Container ID: {}", localID,
+            containerID);
+      }
       pipelineManager.incNumBlocksAllocatedMetric(pipeline.getId());
       return abb.build();
     } catch (PipelineNotFoundException ex) {

+ 7 - 5
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java

@@ -168,11 +168,13 @@ public class SCMBlockDeletingService extends BackgroundService {
             // offline for sometime, the cached commands be flooded.
             eventPublisher.fireEvent(SCMEvents.RETRIABLE_DATANODE_COMMAND,
                 new CommandForDatanode<>(dnId, new DeleteBlocksCommand(dnTXs)));
-            LOG.debug(
-                "Added delete block command for datanode {} in the queue,"
-                    + " number of delete block transactions: {}, TxID list: {}",
-                dnId, dnTXs.size(), String.join(",",
-                    transactions.getTransactionIDList(dnId)));
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                  "Added delete block command for datanode {} in the queue," +
+                      " number of delete block transactions: {}, TxID list: {}",
+                  dnId, dnTXs.size(), String.join(",",
+                      transactions.getTransactionIDList(dnId)));
+            }
           }
         }
         containerManager.updateDeleteTransactionId(transactionMap);

+ 8 - 4
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/command/CommandStatusReportHandler.java

@@ -48,13 +48,17 @@ public class CommandStatusReportHandler implements
     Preconditions.checkNotNull(report);
     List<CommandStatus> cmdStatusList = report.getReport().getCmdStatusList();
     Preconditions.checkNotNull(cmdStatusList);
-    LOGGER.trace("Processing command status report for dn: {}", report
-        .getDatanodeDetails());
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Processing command status report for dn: {}", report
+          .getDatanodeDetails());
+    }
 
     // Route command status to its watchers.
     cmdStatusList.forEach(cmdStatus -> {
-      LOGGER.trace("Emitting command status for id:{} type: {}", cmdStatus
-          .getCmdId(), cmdStatus.getType());
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("Emitting command status for id:{} type: {}", cmdStatus
+            .getCmdId(), cmdStatus.getType());
+      }
       if (cmdStatus.getType() == SCMCommandProto.Type.deleteBlocksCommand) {
         if (cmdStatus.getStatus() == CommandStatus.Status.EXECUTED) {
           publisher.fireEvent(SCMEvents.DELETE_BLOCK_STATUS,

+ 4 - 2
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java

@@ -76,8 +76,10 @@ public class AbstractContainerReportHandler {
         .setSequenceId(replicaProto.getBlockCommitSequenceId())
         .build();
 
-    logger.debug("Processing replica of container {} from datanode {}",
-        containerId, datanodeDetails);
+    if (logger.isDebugEnabled()) {
+      logger.debug("Processing replica of container {} from datanode {}",
+          containerId, datanodeDetails);
+    }
     // Synchronized block should be replaced by container lock,
     // once we have introduced lock inside ContainerInfo.
     synchronized (containerManager.getContainer(containerId)) {

+ 4 - 2
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerActionsHandler.java

@@ -48,8 +48,10 @@ public class ContainerActionsHandler implements
       ContainerID containerId = ContainerID.valueof(action.getContainerID());
       switch (action.getAction()) {
       case CLOSE:
-        LOG.debug("Closing container {} in datanode {} because the" +
-            " container is {}.", containerId, dd, action.getReason());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Closing container {} in datanode {} because the" +
+              " container is {}.", containerId, dd, action.getReason());
+        }
         publisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerId);
         break;
       default:

+ 3 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java

@@ -305,7 +305,9 @@ public class ContainerStateManager {
     pipelineManager.addContainerToPipeline(pipeline.getId(),
         ContainerID.valueof(containerID));
     containerStateCount.incrementAndGet(containerInfo.getState());
-    LOG.trace("New container allocated: {}", containerInfo);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("New container allocated: {}", containerInfo);
+    }
     return containerInfo;
   }
 

+ 4 - 2
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java

@@ -54,8 +54,10 @@ public class IncrementalContainerReportHandler extends
   @Override
   public void onMessage(final IncrementalContainerReportFromDatanode report,
                         final EventPublisher publisher) {
-    LOG.debug("Processing incremental container report from data node {}",
-            report.getDatanodeDetails().getUuid());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing incremental container report from data node {}",
+          report.getDatanodeDetails().getUuid());
+    }
 
     boolean success = true;
     for (ContainerReplicaProto replicaProto :

+ 4 - 2
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackAware.java

@@ -272,8 +272,10 @@ public final class SCMContainerPlacementRackAware extends SCMCommonPolicy {
             " excludedNodes and affinityNode constrains.", null);
       }
       if (hasEnoughSpace((DatanodeDetails)node, sizeRequired)) {
-        LOG.debug("Datanode {} is chosen for container. Required size is {}",
-            node.toString(), sizeRequired);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Datanode {} is chosen for container. Required size is {}",
+              node.toString(), sizeRequired);
+        }
         metrics.incrDatanodeChooseSuccessCount();
         if (isFallbacked) {
           metrics.incrDatanodeChooseFallbackCount();

+ 16 - 6
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerAttribute.java

@@ -167,7 +167,9 @@ public class ContainerAttribute<T> {
     if (attributeMap.containsKey(key)) {
       attributeMap.get(key).clear();
     } else {
-      LOG.debug("key: {} does not exist in the attributeMap", key);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("key: {} does not exist in the attributeMap", key);
+      }
     }
   }
 
@@ -183,13 +185,17 @@ public class ContainerAttribute<T> {
 
     if (attributeMap.containsKey(key)) {
       if (!attributeMap.get(key).remove(value)) {
-        LOG.debug("ContainerID: {} does not exist in the set pointed by " +
-            "key:{}", value, key);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("ContainerID: {} does not exist in the set pointed by " +
+              "key:{}", value, key);
+        }
         return false;
       }
       return true;
     } else {
-      LOG.debug("key: {} does not exist in the attributeMap", key);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("key: {} does not exist in the attributeMap", key);
+      }
       return false;
     }
   }
@@ -206,7 +212,9 @@ public class ContainerAttribute<T> {
     if (this.attributeMap.containsKey(key)) {
       return Collections.unmodifiableNavigableSet(this.attributeMap.get(key));
     }
-    LOG.debug("No such Key. Key {}", key);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("No such Key. Key {}", key);
+    }
     return EMPTY_SET;
   }
 
@@ -237,7 +245,9 @@ public class ContainerAttribute<T> {
       LOG.error("error in update.", ex);
       if (removed) {
         insert(currentKey, value);
-        LOG.trace("reinserted the removed key. {}", currentKey);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("reinserted the removed key. {}", currentKey);
+        }
       }
       throw ex;
     }

+ 4 - 2
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java

@@ -308,8 +308,10 @@ public class ContainerStateMap {
         // be in an inconsistent state,
 
         lifeCycleStateMap.update(currentState, newState, containerID);
-        LOG.trace("Updated the container {} to new state. Old = {}, new = " +
-            "{}", containerID, currentState, newState);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Updated the container {} to new state. Old = {}, new = " +
+              "{}", containerID, currentState, newState);
+        }
 
         // Just flush both old and new data sets from the result cache.
         flushCache(currentInfo);

+ 7 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java

@@ -267,8 +267,10 @@ public class SCMNodeManager implements NodeManager {
       processNodeReport(datanodeDetails, nodeReport);
       LOG.info("Registered Data node : {}", datanodeDetails);
     } catch (NodeAlreadyExistsException e) {
-      LOG.trace("Datanode is already registered. Datanode: {}",
-          datanodeDetails.toString());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Datanode is already registered. Datanode: {}",
+            datanodeDetails.toString());
+      }
     }
 
     return RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success)
@@ -642,7 +644,9 @@ public class SCMNodeManager implements NodeManager {
     List<String> resolvedHosts = dnsToSwitchMapping.resolve(hosts);
     if (resolvedHosts != null && !resolvedHosts.isEmpty()) {
       String location = resolvedHosts.get(0);
-      LOG.debug("Resolve datanode {} return location {}", hostname, location);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Resolve datanode {} return location {}", hostname, location);
+      }
       return location;
     } else {
       LOG.error("Node {} Resolution failed. Please make sure that DNS table " +

+ 3 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java

@@ -74,7 +74,9 @@ public class PipelineReportHandler implements
         pipelineReportFromDatanode.getReport();
     Preconditions.checkNotNull(dn, "Pipeline Report is "
         + "missing DatanodeDetails.");
-    LOGGER.trace("Processing pipeline report for dn: {}", dn);
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Processing pipeline report for dn: {}", dn);
+    }
     for (PipelineReport report : pipelineReport.getPipelineReportList()) {
       try {
         processPipelineReport(report, dn);

+ 3 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java

@@ -190,7 +190,9 @@ public class RatisPipelineProvider implements PipelineProvider {
 
   protected void initializePipeline(Pipeline pipeline) throws IOException {
     final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
-    LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
+    }
     callRatisRpc(pipeline.getNodes(),
         (raftClient, peer) -> {
           RaftClientReply reply = raftClient.groupAdd(group, peer.getId());

+ 3 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java

@@ -59,7 +59,9 @@ public final class RatisPipelineUtils {
   static void destroyPipeline(Pipeline pipeline, Configuration ozoneConf,
       GrpcTlsConfig grpcTlsConfig) {
     final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
-    LOG.debug("destroying pipeline:{} with {}", pipeline.getId(), group);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("destroying pipeline:{} with {}", pipeline.getId(), group);
+    }
     for (DatanodeDetails dn : pipeline.getNodes()) {
       try {
         destroyPipeline(dn, pipeline.getId(), ozoneConf, grpcTlsConfig);

+ 10 - 8
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java

@@ -494,10 +494,12 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
    */
   private void loginAsSCMUser(Configuration conf)
       throws IOException, AuthenticationException {
-    LOG.debug("Ozone security is enabled. Attempting login for SCM user. "
-            + "Principal: {}, keytab: {}",
-        conf.get(HDDS_SCM_KERBEROS_PRINCIPAL_KEY),
-        conf.get(HDDS_SCM_KERBEROS_KEYTAB_FILE_KEY));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Ozone security is enabled. Attempting login for SCM user. "
+              + "Principal: {}, keytab: {}",
+          conf.get(HDDS_SCM_KERBEROS_PRINCIPAL_KEY),
+          conf.get(HDDS_SCM_KERBEROS_KEYTAB_FILE_KEY));
+    }
 
     if (SecurityUtil.getAuthenticationMethod(conf).equals(
         AuthenticationMethod.KERBEROS)) {
@@ -697,10 +699,10 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
                       ContainerStat stat = removalNotification.getValue();
                       // remove invalid container report
                       metrics.decrContainerStat(stat);
-                      LOG.debug(
-                          "Remove expired container stat entry for datanode: " +
-                              "{}.",
-                          removalNotification.getKey());
+                      if (LOG.isDebugEnabled()) {
+                        LOG.debug("Remove expired container stat entry for " +
+                            "datanode: {}.", removalNotification.getKey());
+                      }
                     }
                   }
                 })