RATIS-2412. NettyClient LoggingHandler not working.#1353
Open
slfan1989 wants to merge 1 commit intoapache:masterfrom
Open
RATIS-2412. NettyClient LoggingHandler not working.#1353slfan1989 wants to merge 1 commit intoapache:masterfrom
slfan1989 wants to merge 1 commit intoapache:masterfrom
Conversation
da79d7f to
6cea561
Compare
Contributor
Author
|
@szetszwo Could you please review this PR? Thanks so much! |
szetszwo
reviewed
Feb 18, 2026
Contributor
There was a problem hiding this comment.
@slfan1989 , thanks for working on this! Good catch on the bug.
- Let's add it in the
initializer, see below. - BTW, it currently uses LogLevel.INFO. Would it print a lot messages? Since this is client, how about changing it to WARN?
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
index a42ddaca8..91f970dd3 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
@@ -52,7 +52,6 @@ public class NettyClient implements Closeable {
() -> channel = new Bootstrap()
.group(group)
.channel(NettyUtils.getSocketChannelClass(group))
- .handler(new LoggingHandler(LogLevel.INFO))
.handler(initializer)
.connect(address)
.sync()
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
index f77096e18..fca0834a0 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
@@ -31,6 +31,8 @@ import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto;
import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto;
import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto;
import org.apache.ratis.protocol.exceptions.AlreadyClosedException;
+import org.apache.ratis.thirdparty.io.netty.handler.logging.LogLevel;
+import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler;
import org.apache.ratis.util.IOUtils;
import org.apache.ratis.util.PeerProxyMap;
import org.apache.ratis.util.ProtoUtils;
@@ -141,9 +143,10 @@ public class NettyRpcProxy implements Closeable {
final ChannelInitializer<SocketChannel> initializer
= new ChannelInitializer<SocketChannel>() {
@Override
- protected void initChannel(SocketChannel ch) throws Exception {
+ protected void initChannel(SocketChannel ch) {
final ChannelPipeline p = ch.pipeline();
+ p.addLast(new LoggingHandler(LogLevel.WARN));
p.addLast(new ProtobufVarint32FrameDecoder());
p.addLast(new ProtobufDecoder(RaftNettyServerReplyProto.getDefaultInstance()));
p.addLast(new ProtobufVarint32LengthFieldPrepender());
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
What changes were proposed in this pull request?
Root Cause
The
Bootstrap.handler()method stores the handler in a single field. Each call overwrites the previous value instead of appending. As a result, only the last handler (initializer) is applied, and LoggingHandler is completely ignored.Solution
Combine both handlers into a single ChannelInitializer. This combined initializer first adds the LoggingHandler to the pipeline, then adds the original initializer which will be automatically processed by Netty to initialize the remaining handlers.
What is the link to the Apache JIRA
RATIS-2412. NettyClient LoggingHandler not working.
How was this patch tested?