This is an automated email from the ASF dual-hosted git repository.

tkalkirill pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 5381cbfbe5 IGNITE-22869 Fix some NetworkMessage logging (#4162)
5381cbfbe5 is described below

commit 5381cbfbe549fa83d5788037d82f131ea4cfb711
Author: Kirill Tkalenko <[email protected]>
AuthorDate: Wed Jul 31 18:10:16 2024 +0300

    IGNITE-22869 Fix some NetworkMessage logging (#4162)
---
 .../ignite/internal/network/NetworkMessage.java    |  9 ++++
 .../network/TrackableNetworkMessageHandler.java    |  4 +-
 .../internal/network/DefaultMessagingService.java  | 50 ++++++++++++++++------
 .../internal/network/message/InvokeRequest.java    |  5 +++
 .../internal/network/message/InvokeResponse.java   |  5 +++
 .../ignite/internal/raft/RaftGroupServiceImpl.java |  6 +--
 6 files changed, 62 insertions(+), 17 deletions(-)

diff --git 
a/modules/network-api/src/main/java/org/apache/ignite/internal/network/NetworkMessage.java
 
b/modules/network-api/src/main/java/org/apache/ignite/internal/network/NetworkMessage.java
index c635b59023..ec4f381896 100644
--- 
a/modules/network-api/src/main/java/org/apache/ignite/internal/network/NetworkMessage.java
+++ 
b/modules/network-api/src/main/java/org/apache/ignite/internal/network/NetworkMessage.java
@@ -71,4 +71,13 @@ public interface NetworkMessage extends Cloneable {
      * Public clone version that is implemented in generated *Impl class.
      */
     NetworkMessage clone();
+
+    /**
+     * Returns a lightweight string representation of a network message for 
logging.
+     *
+     * <p>NOTE: Avoid output sensetive information.</p>
+     */
+    default String toStringForLightLogging() {
+        return getClass().toString();
+    }
 }
diff --git 
a/modules/network-api/src/main/java/org/apache/ignite/internal/network/TrackableNetworkMessageHandler.java
 
b/modules/network-api/src/main/java/org/apache/ignite/internal/network/TrackableNetworkMessageHandler.java
index 1b31d5f7d5..fcd5423c09 100644
--- 
a/modules/network-api/src/main/java/org/apache/ignite/internal/network/TrackableNetworkMessageHandler.java
+++ 
b/modules/network-api/src/main/java/org/apache/ignite/internal/network/TrackableNetworkMessageHandler.java
@@ -77,10 +77,10 @@ public class TrackableNetworkMessageHandler implements 
NetworkMessageHandler {
 
         if (durationMillis > MESSAGING_PROCESSING_LOG_THRESHOLD_MILLIS) {
             LOG.warn(
-                    "Message handling has been too long [duration={}ms, 
message=[{}]]",
+                    "Message handling has been too long [duration={}ms, 
message={}]",
                     durationMillis,
                     // Message may include sensitive data, however it seems 
useful to print full message content while testing.
-                    includeSensitive() ? message : message.getClass()
+                    LOG.isDebugEnabled() && includeSensitive() ? message : 
message.toStringForLightLogging()
             );
         }
     }
diff --git 
a/modules/network/src/main/java/org/apache/ignite/internal/network/DefaultMessagingService.java
 
b/modules/network/src/main/java/org/apache/ignite/internal/network/DefaultMessagingService.java
index 5f035e4c7e..d9dd8e9cfb 100644
--- 
a/modules/network/src/main/java/org/apache/ignite/internal/network/DefaultMessagingService.java
+++ 
b/modules/network/src/main/java/org/apache/ignite/internal/network/DefaultMessagingService.java
@@ -21,6 +21,7 @@ import static 
java.util.concurrent.CompletableFuture.failedFuture;
 import static 
org.apache.ignite.internal.network.NettyBootstrapFactory.isInNetworkThread;
 import static 
org.apache.ignite.internal.network.serialization.PerSessionSerializationService.createClassDescriptorsMessages;
 import static 
org.apache.ignite.internal.thread.ThreadOperation.NOTHING_ALLOWED;
+import static 
org.apache.ignite.internal.tostring.IgniteToStringBuilder.includeSensitive;
 import static 
org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
 import static org.apache.ignite.internal.util.IgniteUtils.safeAbs;
 
@@ -383,7 +384,9 @@ public class DefaultMessagingService extends 
AbstractMessagingService {
             return;
         }
 
-        if (inNetworkObject.message() instanceof InvokeResponse) {
+        NetworkMessage message = inNetworkObject.message();
+
+        if (message instanceof InvokeResponse) {
             Executor executor = chooseExecutorInInboundPool(inNetworkObject);
             executor.execute(() -> handleInvokeResponse(inNetworkObject));
             return;
@@ -391,12 +394,12 @@ public class DefaultMessagingService extends 
AbstractMessagingService {
 
         NetworkMessage payload;
         Long correlationId = null;
-        if (inNetworkObject.message() instanceof InvokeRequest) {
-            InvokeRequest invokeRequest = (InvokeRequest) 
inNetworkObject.message();
+        if (message instanceof InvokeRequest) {
+            InvokeRequest invokeRequest = (InvokeRequest) message;
             payload = invokeRequest.message();
             correlationId = invokeRequest.correlationId();
         } else {
-            payload = inNetworkObject.message();
+            payload = message;
         }
 
         Iterator<HandlerContext> handlerContexts = 
getHandlerContexts(payload.groupType()).iterator();
@@ -418,17 +421,29 @@ public class DefaultMessagingService extends 
AbstractMessagingService {
                 logAndRethrowIfError(inNetworkObject, e);
             } finally {
                 long tookMillis = 
TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startedNanos);
+
                 if (tookMillis > 100) {
-                    LOG.warn("Processing of {} from {} took {} ms", 
inNetworkObject.message(), inNetworkObject.sender(), tookMillis);
+                    LOG.warn(
+                            "Processing of {} from {} took {} ms",
+                            LOG.isDebugEnabled() && includeSensitive() ? 
message : message.toStringForLightLogging(),
+                            inNetworkObject.sender(),
+                            tookMillis
+                    );
                 }
             }
         });
     }
 
     private static void logMessageSkipDueToSenderLeft(InNetworkObject 
inNetworkObject) {
-        LOG.info("Sender ID {} ({}) is stale, so skipping message handling: 
{}",
-                inNetworkObject.launchId(), inNetworkObject.consistentId(), 
inNetworkObject.message()
-        );
+        if (LOG.isInfoEnabled()) {
+            NetworkMessage message = inNetworkObject.message();
+
+            LOG.info("Sender ID {} ({}) is stale, so skipping message 
handling: {}",
+                    inNetworkObject.launchId(),
+                    inNetworkObject.consistentId(),
+                    LOG.isDebugEnabled() && includeSensitive() ? message : 
message.toStringForLightLogging()
+            );
+        }
     }
 
     private boolean senderIdIsStale(InNetworkObject obj) {
@@ -515,11 +530,22 @@ public class DefaultMessagingService extends 
AbstractMessagingService {
     }
 
     private static void logAndRethrowIfError(InNetworkObject obj, Throwable e) 
{
-        if (e instanceof UnresolvableConsistentIdException && obj.message() 
instanceof InvokeRequest) {
-            LOG.info("onMessage() failed while processing {} from {} as the 
sender has left the topology",
-                    obj.message(), obj.sender());
+        NetworkMessage message = obj.message();
+
+        if (e instanceof UnresolvableConsistentIdException && message 
instanceof InvokeRequest) {
+            if (LOG.isInfoEnabled()) {
+                LOG.info(
+                        "onMessage() failed while processing {} from {} as the 
sender has left the topology",
+                        LOG.isDebugEnabled() && includeSensitive() ? message : 
message.toStringForLightLogging(),
+                        obj.sender()
+                );
+            }
         } else {
-            LOG.error("onMessage() failed while processing {} from {}", e, 
obj.message(), obj.sender());
+            LOG.error(
+                    "onMessage() failed while processing {} from {}",
+                    e,
+                    LOG.isDebugEnabled() && includeSensitive() ? message : 
message.toStringForLightLogging(), obj.sender()
+            );
         }
 
         if (e instanceof Error) {
diff --git 
a/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeRequest.java
 
b/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeRequest.java
index da5ee20102..a0bf67721f 100644
--- 
a/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeRequest.java
+++ 
b/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeRequest.java
@@ -34,4 +34,9 @@ public interface InvokeRequest extends NetworkMessage {
 
     /** Request's message. */
     NetworkMessage message();
+
+    @Override
+    default String toStringForLightLogging() {
+        return getClass() + ": [correlationId=" + correlationId() + ", 
message=" + message().getClass() + ']';
+    }
 }
diff --git 
a/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeResponse.java
 
b/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeResponse.java
index 73c68e0eee..a73cf9b41b 100644
--- 
a/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeResponse.java
+++ 
b/modules/network/src/main/java/org/apache/ignite/internal/network/message/InvokeResponse.java
@@ -34,4 +34,9 @@ public interface InvokeResponse extends NetworkMessage {
 
     /** Response's message. */
     NetworkMessage message();
+
+    @Override
+    default String toStringForLightLogging() {
+        return getClass() + ": [correlationId=" + correlationId() + ", 
message=" + message().getClass() + ']';
+    }
 }
diff --git 
a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
 
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
index d34e013230..78c287d045 100644
--- 
a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
+++ 
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.raft;
 import static java.lang.System.currentTimeMillis;
 import static java.util.concurrent.ThreadLocalRandom.current;
 import static java.util.stream.Collectors.toList;
+import static 
org.apache.ignite.internal.tostring.IgniteToStringBuilder.includeSensitive;
 import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
 import static org.apache.ignite.raft.jraft.rpc.CliRequests.AddLearnersRequest;
 import static org.apache.ignite.raft.jraft.rpc.CliRequests.AddPeerRequest;
@@ -63,7 +64,6 @@ import 
org.apache.ignite.internal.raft.configuration.RaftConfiguration;
 import org.apache.ignite.internal.raft.service.LeaderWithTerm;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
 import org.apache.ignite.internal.replicator.ReplicationGroupId;
-import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.network.ClusterNode;
@@ -559,7 +559,7 @@ public class RaftGroupServiceImpl implements 
RaftGroupService {
                     .whenComplete((resp, err) -> {
                         if (LOG.isTraceEnabled()) {
                             LOG.trace("sendWithRetry resp={} from={} to={} 
err={}",
-                                    S.toString(resp),
+                                    resp,
                                     
cluster.topologyService().localMember().address(),
                                     peer.consistentId(),
                                     err == null ? null : err.getMessage());
@@ -597,7 +597,7 @@ public class RaftGroupServiceImpl implements 
RaftGroupService {
                     "Recoverable error during the request occurred (will be 
retried on the randomly selected node) "
                             + "[request={}, peer={}, newPeer={}].",
                     err,
-                    sentRequest,
+                    LOG.isDebugEnabled() && includeSensitive() ? sentRequest : 
sentRequest.toStringForLightLogging(),
                     peer,
                     randomPeer
             );

Reply via email to