Flavio, Thanks for your quick response. > bq. alternatively reader may try to read ack quorum reads for LAC+1
> yeah, that isn't really safe. Are you worried about the case where reader may read ahead of writer got Acks and crashes before returning to the user? I agree this solution does have that corner case issue. > My concern here is that it breaks the flow of the write pipeline. but other than that, I don't see a correctness issue. In theory yes. But this happens only when writes are sparse i.e if there is a write pipeline we won't even come to this case right? Thanks, JV On Wed, Sep 23, 2015 at 8:37 AM, Flavio Junqueira (JIRA) <[email protected]> wrote: > > [ > https://issues.apache.org/jira/browse/BOOKKEEPER-874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14904675#comment-14904675 > ] > > Flavio Junqueira commented on BOOKKEEPER-874: > --------------------------------------------- > > bq. Can't we update LAC of the last entry in place on bookies? Does this > violate any BookKeeper's principles? > > My concern here is that it breaks the flow of the write pipeline. but > other than that, I don't see a correctness issue. > > bq. alternatively reader may try to read ack quorum reads for LAC+1 > > yeah, that isn't really safe. > > > Explict LAC from Writer to Bookies > > ---------------------------------- > > > > Key: BOOKKEEPER-874 > > URL: > https://issues.apache.org/jira/browse/BOOKKEEPER-874 > > Project: Bookkeeper > > Issue Type: Improvement > > Components: bookkeeper-client, bookkeeper-server > > Reporter: Venkateswararao Jujjuri > > Assignee: Venkateswararao Jujjuri > > > > Current client API piggy-backs LAC with a write. This is keeps reader > one behind the writer. In order to keep reader up to date with writer even > when there is a pause in write, proposing the following: > > Writer sends explicit LAC on a configured timeout if there is no write > within that period. > > > > -- > This message was sent by Atlassian JIRA > (v6.3.4#6332) > -- Jvrao --- First they ignore you, then they laugh at you, then they fight you, then you win. - Mahatma Gandhi
