Skip to content

Commit

Permalink
[hotfix][network] Remove the deduplication for CancelPartitionRequest.
Browse files Browse the repository at this point in the history
To avoid the memory leak in the record map for deduplication.
  • Loading branch information
KarmaGYZ committed Feb 16, 2022
1 parent 3106f14 commit c0cca2e
Showing 1 changed file with 1 addition and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -67,13 +67,6 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap
private final ChannelFutureListener writeListener =
new WriteAndFlushNextMessageIfPossibleListener();

/**
* Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
* while data is still coming in for this channel.
*/
private final ConcurrentMap<InputChannelID, InputChannelID> cancelled =
new ConcurrentHashMap<>();

/**
* The channel handler context is initialized in channel active event by netty thread, the
* context may also be accessed by task thread or canceler thread to cancel partition request
Expand Down Expand Up @@ -108,9 +101,7 @@ public void cancelRequestFor(InputChannelID inputChannelId) {
return;
}

if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
}
ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
}

// ------------------------------------------------------------------------
Expand Down

0 comments on commit c0cca2e

Please sign in to comment.