Skip to content

Commit 8d85f67

Browse files
authored
[client] Check whether request is removed from inflight requests to avoid NullPointerException when connection is closed. (#2127)
1 parent 58957fd commit 8d85f67

File tree

1 file changed

+9
-1
lines changed

1 file changed

+9
-1
lines changed

fluss-rpc/src/main/java/org/apache/fluss/rpc/netty/client/ServerConnection.java

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -166,7 +166,9 @@ private CompletableFuture<Void> close(Throwable cause) {
166166
// notify all the inflight requests
167167
for (int requestId : inflightRequests.keySet()) {
168168
InflightRequest request = inflightRequests.remove(requestId);
169-
request.responseFuture.completeExceptionally(requestCause);
169+
if (request != null) {
170+
request.responseFuture.completeExceptionally(requestCause);
171+
}
170172
}
171173

172174
// notify all the pending requests
@@ -249,6 +251,12 @@ public void onFailure(Throwable cause) {
249251
private void establishConnection(ChannelFuture future, boolean isInnerClient) {
250252
synchronized (lock) {
251253
if (future.isSuccess()) {
254+
if (state.isDisconnected()) {
255+
LOG.debug(
256+
"Connection established to {} but connection is already closed.", node);
257+
future.channel().close();
258+
return;
259+
}
252260
LOG.debug("Established connection to server {}.", node);
253261
channel = future.channel();
254262
channel.pipeline()

0 commit comments

Comments
 (0)