[
https://issues.apache.org/jira/browse/BOOKKEEPER-620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13688825#comment-13688825
]
Sijie Guo commented on BOOKKEEPER-620:
--------------------------------------
{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.
{quote}
that's not true. new requests would be failed directly since the channel is
closed, writing a request to a closed request would error out the request
directly. error outing a request would trigger replacing a new bookie. that is
not a big deal.
but if you set the channel to DISCONNECTED first, you would error out outgoing
requests. that is my concern.
{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}
{quote}
- // DO NOT rewrite these using Map.Entry iterations. We want to iterate
- // on keys and see if we are successfully able to remove the key from
- // the map. Because the add and the read methods also do the same thing
- // in case they get a write failure on the socket. The one who
- // successfully removes the key from the map is the one responsible for
- // calling the application callback.
{quote}
you just moved around the code, which doesn't change the behavior how we
iterate the keys. the comment is about iterating keys. so I don't think it is
redundant as you said.
> 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