RATIS-1853. When the stream server channelInactive, all requests in the channel.#889
Conversation
szetszwo
left a comment
There was a problem hiding this comment.
@guohao-rosicky , thanks a lot for working on this! Please see the comments inlined. Most of them are cosmetic changes.
| } | ||
| } | ||
|
|
||
| public static class RequestTracker { |
There was a problem hiding this comment.
Let's call it ChannelMap.
| } | ||
|
|
||
| public static class RequestTracker { | ||
| private final Map<ChannelId, Set<ClientInvocationId>> ref = new ConcurrentHashMap<>(); |
There was a problem hiding this comment.
Since the field is only used locally in this class, let's simply call it map.
|
|
||
| public void add(ChannelId channelId, | ||
| ClientInvocationId clientInvocationId) { | ||
| ref.computeIfAbsent(channelId, (e) -> new CopyOnWriteArraySet<>()).add(clientInvocationId); |
There was a problem hiding this comment.
ConcurrentHashMap is more efficient than CopyOnWriteArraySet. Let's declare it as
private final Map<ChannelId, Map<ClientInvocationId, ClientInvocationId>> map = new ConcurrentHashMap<>();and use ConcurrentHashMap.
| public Set<ClientInvocationId> removeAll(ChannelId channelId) { | ||
| return ref.remove(channelId); | ||
| } |
There was a problem hiding this comment.
Let's simply call it remove.
Set<ClientInvocationId> remove(ChannelId channelId) {
return Optional.ofNullable(map.remove(channelId))
.map(Map::keySet)
.orElse(Collections.emptySet());
}|
|
||
| void cleanUpOnChannelInactive(ClientInvocationId key) { | ||
| Optional.ofNullable(streams.remove(key)).ifPresent(removed -> removed.getLocal().cleanUp()); | ||
| void cleanUpOnChannelInactive(ChannelHandlerContext ctx, TimeDuration channelInactiveGracePeriod) { |
There was a problem hiding this comment.
Let's pass ChannelId instead of the ChannelHandlerContext.
| // Delayed memory garbage cleanup | ||
| Optional.ofNullable(requestTracker.removeAll(ctx.channel().id())).ifPresent(ids -> { | ||
| final String streamIds = Arrays.toString(ids.toArray()); | ||
| LOG.info("server {} is disconnect, cleanup clientInvocationIds={}", ctx.channel(), streamIds); |
There was a problem hiding this comment.
Let's use the same wording.
LOG.info("Channel {} is inactive, cleanup clientInvocationIds={}", channelId, ids);| () -> { | ||
| for (ClientInvocationId clientInvocationId : ids) { | ||
| Optional.ofNullable(streams.remove(clientInvocationId)).ifPresent(removed -> | ||
| removed.getLocal().cleanUp()); | ||
| } | ||
| }, |
There was a problem hiding this comment.
It will be easier to read if it is refactored out
void cleanUp(Set<ClientInvocationId> ids) {
for (ClientInvocationId clientInvocationId : ids) {
Optional.ofNullable(streams.remove(clientInvocationId))
.map(StreamInfo::getLocal)
.ifPresent(LocalStream::cleanUp);
}
}
void cleanUpOnChannelInactive(ChannelId channelId, TimeDuration channelInactiveGracePeriod) {
// Delayed memory garbage cleanup
Optional.ofNullable(channels.remove(channelId)).ifPresent(ids -> {
LOG.info("Channel {} is inactive, cleanup clientInvocationIds={}", channelId, ids);
TimeoutExecutor.getInstance().onTimeout(channelInactiveGracePeriod, () -> cleanUp(ids),
LOG, () -> "Timeout check failed, clientInvocationIds=" + ids);
});
}|
Thanks for the review @szetszwo . |
szetszwo
left a comment
There was a problem hiding this comment.
+1 the change looks good.
What changes were proposed in this pull request?
When the stream server channelInactive, all requests in the channel.
At the same time, there will be multiple stream sessions, using the same channel, for transmission, and when the connection is broken, we should clean up more stream sessions
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-1853