[
https://issues.apache.org/jira/browse/BOOKKEEPER-620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13686555#comment-13686555
]
Ivan Kelly commented on BOOKKEEPER-620:
---------------------------------------
New patch addresses all comments except the channelDisconnected one.
{quote}
1) in general, it would be better to keep comments when you moved around the
code.
{quote}
Both comment blocks on #errorOutOutstandingEntries() are redundant after the
move. The method doc talks about classes that were removed from the codebase
before I started working on it.
{quote}
2) it would be better to error out entries before set state back to
DISCONNECTED. since after the channel is set to DISCONNECTED, the client would
begin sending requests to connecting. you might error ongoing requests.
{quote}
If we error out outstanding requests before setting state to DISCONNECTED,
there is a time period between the erroring and the synchronization block where
more requests could be added. These would not be errored. We can't error out
inside the synchronization block because it could lead to a deadlock if the
request was retried. As it is now, it won't touch ongoing requests, as we only
take the set of requests to error at the same time as setting the state to
disconnected.
> PerChannelBookieClient race during channel disconnect
> -----------------------------------------------------
>
> Key: BOOKKEEPER-620
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-620
> Project: Bookkeeper
> Issue Type: Bug
> Components: bookkeeper-client
> Reporter: Sijie Guo
> Assignee: Ivan Kelly
> Fix For: 4.2.2, 4.3.0
>
> Attachments:
> 0001-BOOKKEEPER-620-PerChannelBookieClient-race-during-ch.patch,
> 0001-BOOKKEEPER-620-PerChannelBookieClient-race-during-ch.patch,
> 0001-BOOKKEEPER-620-PerChannelBookieClient-race-during-ch.patch,
> 0002-BOOKKEEPER-620-PerChannelBookieClient-race-during-ch.patch,
> BOOKKEEPER-620.diff
>
>
> channel & state are not synchronized in PerChannelBookieClient#closeInternal.
> so it might cause state is set to CONNECTED but the netty channel is closed
> by mistake in closeInternal.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira