You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jason Gustafson <ja...@confluent.io> on 2020/04/16 23:44:59 UTC

[DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Hi All,

I'd like to start a discussion on KIP-595:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum.
This proposal specifies a Raft protocol to ultimately replace Zookeeper as
documented in KIP-500. Please take a look and share your thoughts.

A few minor notes to set the stage a little bit:

- This KIP does not specify the structure of the messages used to represent
metadata in Kafka, nor does it specify the internal API that will be used
by the controller. Expect these to come in later proposals. Here we are
primarily concerned with the replication protocol and basic operational
mechanics.
- We expect many details to change as we get closer to integration with
the controller. Any changes we make will be made either as amendments to
this KIP or, in the case of larger changes, as new proposals.
- We have a prototype implementation which I will put online within the
next week which may help in understanding some details. It has diverged a
little bit from our proposal, so I am taking a little time to bring it in
line. I'll post an update to this thread when it is available for review.

Finally, I want to mention that this proposal was drafted by myself, Boyang
Chen, and Guozhang Wang.

Thanks,
Jason

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jason Gustafson <ja...@confluent.io>.
As promised, here is a link to the current prototype:
https://github.com/confluentinc/kafka/tree/kafka-raft.

Thanks,
Jason

On Mon, Apr 20, 2020 at 10:53 AM Jason Gustafson <ja...@confluent.io> wrote:

> Hi Deng,
>
> Thanks for the question. I mentioned this in the rejected alternatives
> section. The current proposal is only for metadata, but I am definitely in
> favor of using Raft for partition replication in the long term as well.
> There are some interesting tradeoffs in terms of fault tolerance, latency,
> and batching compared with the current replication protocol. I consider
> this a good candidate for the next big architectural change once KIP-500
> nears completion.
>
> -Jason
>
> On Sun, Apr 19, 2020 at 7:16 PM deng ziming <de...@gmail.com>
> wrote:
>
>> Big +1 for your initiative and I have a question, we implement the
>> Raft protocol
>> just to be used in the management of metadata in Zookeeper or we will also
>> use it to replace the current logical of managing log-replica since the
>> algorithm we used to manage log-replica is analogous to Raft.
>>
>>
>> On Fri, Apr 17, 2020 at 7:45 AM Jason Gustafson <ja...@confluent.io>
>> wrote:
>>
>> > Hi All,
>> >
>> > I'd like to start a discussion on KIP-595:
>> >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
>> > .
>> > This proposal specifies a Raft protocol to ultimately replace Zookeeper
>> as
>> > documented in KIP-500. Please take a look and share your thoughts.
>> >
>> > A few minor notes to set the stage a little bit:
>> >
>> > - This KIP does not specify the structure of the messages used to
>> represent
>> > metadata in Kafka, nor does it specify the internal API that will be
>> used
>> > by the controller. Expect these to come in later proposals. Here we are
>> > primarily concerned with the replication protocol and basic operational
>> > mechanics.
>> > - We expect many details to change as we get closer to integration with
>> > the controller. Any changes we make will be made either as amendments to
>> > this KIP or, in the case of larger changes, as new proposals.
>> > - We have a prototype implementation which I will put online within the
>> > next week which may help in understanding some details. It has diverged
>> a
>> > little bit from our proposal, so I am taking a little time to bring it
>> in
>> > line. I'll post an update to this thread when it is available for
>> review.
>> >
>> > Finally, I want to mention that this proposal was drafted by myself,
>> Boyang
>> > Chen, and Guozhang Wang.
>> >
>> > Thanks,
>> > Jason
>> >
>>
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Deng,

Thanks for the question. I mentioned this in the rejected alternatives
section. The current proposal is only for metadata, but I am definitely in
favor of using Raft for partition replication in the long term as well.
There are some interesting tradeoffs in terms of fault tolerance, latency,
and batching compared with the current replication protocol. I consider
this a good candidate for the next big architectural change once KIP-500
nears completion.

-Jason

On Sun, Apr 19, 2020 at 7:16 PM deng ziming <de...@gmail.com>
wrote:

> Big +1 for your initiative and I have a question, we implement the
> Raft protocol
> just to be used in the management of metadata in Zookeeper or we will also
> use it to replace the current logical of managing log-replica since the
> algorithm we used to manage log-replica is analogous to Raft.
>
>
> On Fri, Apr 17, 2020 at 7:45 AM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi All,
> >
> > I'd like to start a discussion on KIP-595:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > .
> > This proposal specifies a Raft protocol to ultimately replace Zookeeper
> as
> > documented in KIP-500. Please take a look and share your thoughts.
> >
> > A few minor notes to set the stage a little bit:
> >
> > - This KIP does not specify the structure of the messages used to
> represent
> > metadata in Kafka, nor does it specify the internal API that will be used
> > by the controller. Expect these to come in later proposals. Here we are
> > primarily concerned with the replication protocol and basic operational
> > mechanics.
> > - We expect many details to change as we get closer to integration with
> > the controller. Any changes we make will be made either as amendments to
> > this KIP or, in the case of larger changes, as new proposals.
> > - We have a prototype implementation which I will put online within the
> > next week which may help in understanding some details. It has diverged a
> > little bit from our proposal, so I am taking a little time to bring it in
> > line. I'll post an update to this thread when it is available for review.
> >
> > Finally, I want to mention that this proposal was drafted by myself,
> Boyang
> > Chen, and Guozhang Wang.
> >
> > Thanks,
> > Jason
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by deng ziming <de...@gmail.com>.
Big +1 for your initiative and I have a question, we implement the
Raft protocol
just to be used in the management of metadata in Zookeeper or we will also
use it to replace the current logical of managing log-replica since the
algorithm we used to manage log-replica is analogous to Raft.


On Fri, Apr 17, 2020 at 7:45 AM Jason Gustafson <ja...@confluent.io> wrote:

> Hi All,
>
> I'd like to start a discussion on KIP-595:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> .
> This proposal specifies a Raft protocol to ultimately replace Zookeeper as
> documented in KIP-500. Please take a look and share your thoughts.
>
> A few minor notes to set the stage a little bit:
>
> - This KIP does not specify the structure of the messages used to represent
> metadata in Kafka, nor does it specify the internal API that will be used
> by the controller. Expect these to come in later proposals. Here we are
> primarily concerned with the replication protocol and basic operational
> mechanics.
> - We expect many details to change as we get closer to integration with
> the controller. Any changes we make will be made either as amendments to
> this KIP or, in the case of larger changes, as new proposals.
> - We have a prototype implementation which I will put online within the
> next week which may help in understanding some details. It has diverged a
> little bit from our proposal, so I am taking a little time to bring it in
> line. I'll post an update to this thread when it is available for review.
>
> Finally, I want to mention that this proposal was drafted by myself, Boyang
> Chen, and Guozhang Wang.
>
> Thanks,
> Jason
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by jianjie feng <au...@gmail.com>.
Hello, Jason

  I'd like to attend this project and do some development, could you tell where shall I start ?

  Thanks
  Jianjie

On 2020/04/16 23:44:59, Jason Gustafson <ja...@confluent.io> wrote: 
> Hi All,
> 
> I'd like to start a discussion on KIP-595:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum.
> This proposal specifies a Raft protocol to ultimately replace Zookeeper as
> documented in KIP-500. Please take a look and share your thoughts.
> 
> A few minor notes to set the stage a little bit:
> 
> - This KIP does not specify the structure of the messages used to represent
> metadata in Kafka, nor does it specify the internal API that will be used
> by the controller. Expect these to come in later proposals. Here we are
> primarily concerned with the replication protocol and basic operational
> mechanics.
> - We expect many details to change as we get closer to integration with
> the controller. Any changes we make will be made either as amendments to
> this KIP or, in the case of larger changes, as new proposals.
> - We have a prototype implementation which I will put online within the
> next week which may help in understanding some details. It has diverged a
> little bit from our proposal, so I am taking a little time to bring it in
> line. I'll post an update to this thread when it is available for review.
> 
> Finally, I want to mention that this proposal was drafted by myself, Boyang
> Chen, and Guozhang Wang.
> 
> Thanks,
> Jason
> 

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Tom,

Thanks for reviewing the doc! Some replies inlined.

On Tue, Jul 14, 2020 at 4:11 AM Tom Bentley <tb...@redhat.com> wrote:

> Hi Jason, Ghouzang and Boyang,
>
> First, thanks for the KIP. I've not a few minor suggestions and nits, but
> on the whole it was pretty clear and understandable.
>
> 1. § Motivation
>
>     "We have intentionally avoided any assumption about the representation
> of the log and its semantics."
>
> I found this a bit confusing because there's a whole section about the
> assumed log structure later on. I think something like:
>
>     "We make no assumption about the physical log format and minimal
> assumptions about it's logical structure (see §Log Structure)."
>
> might make things a little clearer, though that suggestion does omit any
> claim about semantics.
>
> Furthermore, in the "Log Structure" section it ends with "Kafka's current
> v2 message format version supports everything we need, so we will assume
> that."
> All you need to say here is that "the v2 message format has the
> assumed/required properties", saying you're assuming the v2 message format
> negates the rest of the paragraph.
>
>
Sounds good, will update.


>
> 2. § Configurations
>
> The format for voters in the `quorum.voters` is potentially a little
> confusing because the separation of the broker id from the broker host name
> with a dot means the whole thing is syntactically indistinguishable from a
> host name. Using a different separator would avoid such ambiguity.
>
>
Hmm that's a good question given a popular host name would be "127.0.0.1".
Maybe we should just use colon as well, like "1:kafka-1:9092" and "2:
127.0.0.1:9093".


>
> 3. § Leader Election and Data Replication
>
>     "The key functionalities of any consensus protocol are leader election
> and data replication. The protocol for these two functionalities consists
> of 5 core RPCs:"
>
> This sentence is followed by a list of 4 (rather than 5) items. I don't
> think you meant to imply DescribeQuorum is a core RPC.
>
> Yes, nice catch. We moved the DescribeQuorum out of core but forgot to
update the other text. Will update.


>
> 4. § BeginQuorumEpoch
>
>     "observers will discover the new leader through either
> the DiscoverBrokers or Fetch APIs"
>
> DiscoverBrokers isn't defined in this KIP (I think it was when I read an
> earlier version), and I don't recall seeing another KIP which describes it.
>
>
Yes, this text also needs to be updated: we originally have a proposal of
DiscoverBrokers but decided to replace it with the log replication
mechanism, I will update it.


>
> 5. § EndQuorumEpoch
>
>     "If the node's priority is highest, it will become candidate
> immediately instead of waiting for next poll."
>
> This is clarified in the later section on EndquorumEpoch request handling,
> but I found the wording " instead of waiting for next poll" to be
> confusing. Maybe "instead of waiting for next election timeout" would be
> clearer?
>
> Ack.


>
> 6. § Fetch
>
> The diagram from the Raft dissertation is a broken link (I'm guessing for
> anyone lacking access to https://confluentinc.atlassian.net).
>
>
Our bad, I will update the link.


>
> 7. § Fetch
>
>     "This proposal also extends the Fetch response to include a separate
> field for the receiver of the request to indicate the current leader."
>
> I think "receiver of the request" should be "receiver of the response",
> shouldn't it?
>
>
It is actually for the receiver of the request indeed. The point is that if
a request is sent to the non-leader, in addition to returning an error code
it would also encode the current leader it knows of (if it knows, of
course) so that the fetcher would not need to send another round of
metadata request discovering it.


>
> 8. § Fetch § Response Schema
>
> The `NextOffsetAndEpoch` and `CurrentLeader` field in the fetch response
> lack `about` properties.
>
> Ack, will update.


>
> 9. § FetchRequest Handling
>
> Item 1.a. is covering only the case where the fetcher is a voter (thus in
> receipt of BeginQuorumEpoch), it doesn't cover observer fetchers.
>
> Also this algorithm talks about "the leader epoch" being larger or smaller,
> but it's not immediately clear whether this is the leader's epoch or the
> leader epoch in the fetch request and which is the epoch being compared
> against. It makes the comparisons harder to understand if you have to guess
> the order of the operands.
>
>
For observers, they will also eventually learn about the leader epoch via
the metadata request; the comparison is between the epoch of the leader
that received the request, and the encoded "CurrentLeaderEpoch" in the
request. I will update the doc.


>
> 10. § Discussion: Replication Progress Timeout for Zombie Leader
>
>     "Note that the node will remain a leader until it finds that it has
> been supplanted by another voter."
>
> I don't quite follow this: Why is a _voter_ involved here? Surely it's
> simply a matter of the leader observing that a new leader exists which
> determines whether it steps down.
>
>
I do not find the corresponding text you referred to in the doc?


>
> 11. § DescribeQuorum
>
> This seems to be defined also in KIP-642.
>
> Ack, I will update the doc.


>
> 12. § Tooling support
>
> `kafka-metadata-quorum.sh`: Given that the protocol is essentially
> independent of the partitions for which the quorum is established
> it seems a little odd to name the tool so that it can't later be extended
> to support quorums for other partitions.
> For example why not `kafka-quorum.sh --metadata --describe` and later
> support `kafka-quorum.sh --topic-partitions foo-0,foo-1 --describe`.
> Obviously you don't need `--metadata` (it could be the default if
> `--topic-partitions` was omitted), it's more a point about the name of the
> tool.
>
> Again in this section, on the one hand the list says `--describe` supports
> two arguments (status and replication).
> On the other hand the example shows a plain, argumentless `--describe`.
>
>
> 13. § Metrics
>
> i. I wonder about including `raft` in the metric names, since it's not
> really exposed elsewhere in the public API.
>

Are you suggesting to remove `raft` from the `type` string value?


>
> ii. Similarly to the tool/script, we end up with metrics which are
> implicitly about a partition. If we use quorums for other partitions in the
> future we'll need to replicate a lot of these metrics, which would make it
> harder to abstract across collections of metrics.
>
>
My thoughts are when raft is used for other topic partitions, then we would
include the partition as a tag in the metric in addition to the `type` and
`metric-name`. Does that work?


>
> Thanks again for the KIP,
>
> Tom
>
>
>
> On Mon, Jul 13, 2020 at 5:50 PM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi All,
> >
> > Just a quick update on the proposal. We have decided to move quorum
> > reassignment to a separate KIP:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > .
> > The way this ties into cluster bootstrapping is complicated, so we felt
> we
> > needed a bit more time for validation. That leaves the core of this
> > proposal as quorum-based replication. If there are no further comments,
> we
> > will plan to start a vote later this week.
> >
> > Thanks,
> > Jason
> >
> > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > @Jun Rao <ju...@gmail.com>
> > >
> > > Regarding your comment about log compaction. After some deep-diving
> into
> > > this we've decided to propose a new snapshot-based log cleaning
> mechanism
> > > which would be used to replace the current compaction mechanism for
> this
> > > meta log. A new KIP will be proposed specifically for this idea.
> > >
> > > All,
> > >
> > > I've updated the KIP wiki a bit updating one config "
> > > election.jitter.max.ms"
> > > to "election.backoff.max.ms" to make it more clear about the usage:
> the
> > > configured value will be the upper bound of the binary exponential
> > backoff
> > > time after a failed election, before starting a new one.
> > >
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> reluctanthero104@gmail.com>
> > > wrote:
> > >
> > > > Thanks for the suggestions Guozhang.
> > > >
> > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hello Boyang,
> > > > >
> > > > > Thanks for the updated information. A few questions here:
> > > > >
> > > > > 1) Should the quorum-file also update to support multi-raft?
> > > > >
> > > > > I'm neutral about this, as we don't know yet how the multi-raft
> > modules
> > > > would behave. If
> > > > we have different threads operating different raft groups,
> > consolidating
> > > > the `checkpoint` files seems
> > > > not reasonable. We could always add `multi-quorum-file` later if
> > > possible.
> > > >
> > > > 2) In the previous proposal, there's fields in the FetchQuorumRecords
> > > like
> > > > > latestDirtyOffset, is that dropped intentionally?
> > > > >
> > > > > I dropped the latestDirtyOffset since it is associated with the log
> > > > compaction discussion. This is beyond this KIP scope and we could
> > > > potentially get a separate KIP to talk about it.
> > > >
> > > >
> > > > > 3) I think we also need to elaborate a bit more details regarding
> > when
> > > to
> > > > > send metadata request and discover-brokers; currently we only
> > discussed
> > > > > during bootstrap how these requests would be sent. I think the
> > > following
> > > > > scenarios would also need these requests
> > > > >
> > > > > 3.a) As long as a broker does not know the current quorum
> (including
> > > the
> > > > > leader and the voters), it should continue periodically ask other
> > > brokers
> > > > > via "metadata.
> > > > > 3.b) As long as a broker does not know all the current quorum
> voter's
> > > > > connections, it should continue periodically ask other brokers via
> > > > > "discover-brokers".
> > > > > 3.c) When the leader's fetch timeout elapsed, it should send
> metadata
> > > > > request.
> > > > >
> > > > > Make sense, will add to the KIP.
> > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey all,
> > > > > >
> > > > > > follow-up on the previous email, we made some more updates:
> > > > > >
> > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > > > > multi-raft.
> > > > > >
> > > > > > 2. We add observer status into the DescribeQuorumResponse as we
> see
> > > it
> > > > > is a
> > > > > > low hanging fruit which is very useful for user debugging and
> > > > > reassignment.
> > > > > >
> > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which
> > is
> > > > > purely
> > > > > > in charge of discovering broker connections in a gossip manner.
> The
> > > > > quorum
> > > > > > leader discovery is piggy-back on the Metadata RPC for the topic
> > > > > partition
> > > > > > leader, which in our case is the single metadata partition for
> the
> > > > > version
> > > > > > one.
> > > > > >
> > > > > > Let me know if you have any questions.
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > reluctanthero104@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey all,
> > > > > > >
> > > > > > > Thanks for the great discussions so far. I'm posting some KIP
> > > updates
> > > > > > from
> > > > > > > our working group discussion:
> > > > > > >
> > > > > > > 1. We will be changing the core RPCs from single-raft API to
> > > > > multi-raft.
> > > > > > > This means all protocols will be "batch" in the first version,
> > but
> > > > the
> > > > > > KIP
> > > > > > > itself only illustrates the design for a single metadata topic
> > > > > partition.
> > > > > > > The reason is to "keep the door open" for future extensions of
> > this
> > > > > piece
> > > > > > > of module such as a sharded controller or general quorum based
> > > topic
> > > > > > > replication, beyond the current Kafka replication protocol.
> > > > > > >
> > > > > > > 2. We will piggy-back on the current Kafka Fetch API instead of
> > > > > inventing
> > > > > > > a new FetchQuorumRecords RPC. The motivation is about the same
> as
> > > #1
> > > > as
> > > > > > > well as making the integration work easier, instead of letting
> > two
> > > > > > similar
> > > > > > > RPCs diverge.
> > > > > > >
> > > > > > > 3. In the EndQuorumEpoch protocol, instead of only sending the
> > > > request
> > > > > to
> > > > > > > the most caught-up voter, we shall broadcast the information to
> > all
> > > > > > voters,
> > > > > > > with a sorted voter list in descending order of their
> > corresponding
> > > > > > > replicated offset. In this way, the top voter will become a
> > > candidate
> > > > > > > immediately, while the other voters shall wait for an
> exponential
> > > > > > back-off
> > > > > > > to trigger elections, which helps ensure the top voter gets
> > > elected,
> > > > > and
> > > > > > > the election eventually happens when the top voter is not
> > > responsive.
> > > > > > >
> > > > > > > Please see the updated KIP and post any questions or concerns
> on
> > > the
> > > > > > > mailing thread.
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > >
> > > > > > >> Hi, Guozhang and Jason,
> > > > > > >>
> > > > > > >> Thanks for the reply. A couple of more replies.
> > > > > > >>
> > > > > > >> 102. Still not sure about this. How is the tombstone issue
> > > addressed
> > > > > in
> > > > > > >> the
> > > > > > >> non-voter and the observer.  They can die at any point and
> > restart
> > > > at
> > > > > an
> > > > > > >> arbitrary later time, and the advancing of the firstDirty
> offset
> > > and
> > > > > the
> > > > > > >> removal of the tombstone can happen independently.
> > > > > > >>
> > > > > > >> 106. I agree that it would be less confusing if we used
> "epoch"
> > > > > instead
> > > > > > of
> > > > > > >> "leader epoch" consistently.
> > > > > > >>
> > > > > > >> Jun
> > > > > > >>
> > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > > >> >
> > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > >> >
> > > > > > >> > > Hi, Guozhang,
> > > > > > >> > >
> > > > > > >> > > Thanks for the reply. A few more replies inlined below.
> > > > > > >> > >
> > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Hello Jun,
> > > > > > >> > > >
> > > > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > > > >> > > >
> > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > > >> > > > > 101.1 Currently, we support auto broker id generation.
> > Is
> > > > this
> > > > > > >> > > supported
> > > > > > >> > > > > for bootstrap brokers?
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > The vote ids would just be the broker ids.
> > > "bootstrap.servers"
> > > > > > >> would be
> > > > > > >> > > > similar to what client configs have today, where
> > > > "quorum.voters"
> > > > > > >> would
> > > > > > >> > be
> > > > > > >> > > > pre-defined config values.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > My question was on the auto generated broker id.
> Currently,
> > > the
> > > > > > broker
> > > > > > >> > can
> > > > > > >> > > choose to have its broker Id auto generated. The
> generation
> > is
> > > > > done
> > > > > > >> > through
> > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how
> > the
> > > > > > broker
> > > > > > >> id
> > > > > > >> > is
> > > > > > >> > > auto generated. "quorum.voters" also can't be set
> statically
> > > if
> > > > > > broker
> > > > > > >> > ids
> > > > > > >> > > are auto generated.
> > > > > > >> > >
> > > > > > >> > > Jason has explained some ideas that we've discussed so
> far,
> > > the
> > > > > > >> reason we
> > > > > > >> > intentional did not include them so far is that we feel it
> is
> > > > > out-side
> > > > > > >> the
> > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> > > > definitely
> > > > > > >> > address them though.
> > > > > > >> >
> > > > > > >> > On the high-level, our belief is that "joining a quorum" and
> > > > > "joining
> > > > > > >> (or
> > > > > > >> > more specifically, registering brokers in) the cluster"
> would
> > be
> > > > > > >> > de-coupled a bit, where the former should be completed
> before
> > we
> > > > do
> > > > > > the
> > > > > > >> > latter. More specifically, assuming the quorum is already up
> > and
> > > > > > >> running,
> > > > > > >> > after the newly started broker found the leader of the
> quorum
> > it
> > > > can
> > > > > > >> send a
> > > > > > >> > specific RegisterBroker request including its listener /
> > > protocol
> > > > /
> > > > > > etc,
> > > > > > >> > and upon handling it the leader can send back the uniquely
> > > > generated
> > > > > > >> broker
> > > > > > >> > id to the new broker, while also executing the
> > "startNewBroker"
> > > > > > >> callback as
> > > > > > >> > the controller.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >
> > > > > > >> > > > > 102. Log compaction. One weak spot of log compaction
> is
> > > for
> > > > > the
> > > > > > >> > > consumer
> > > > > > >> > > > to
> > > > > > >> > > > > deal with deletes. When a key is deleted, it's
> retained
> > > as a
> > > > > > >> > tombstone
> > > > > > >> > > > > first and then physically removed. If a client misses
> > the
> > > > > > >> tombstone
> > > > > > >> > > > > (because it's physically removed), it may not be able
> to
> > > > > update
> > > > > > >> its
> > > > > > >> > > > > metadata properly. The way we solve this in Kafka is
> > based
> > > > on
> > > > > a
> > > > > > >> > > > > configuration (log.cleaner.delete.retention.ms) and
> we
> > > > > expect a
> > > > > > >> > > consumer
> > > > > > >> > > > > having seen an old key to finish reading the deletion
> > > > > tombstone
> > > > > > >> > within
> > > > > > >> > > > that
> > > > > > >> > > > > time. There is no strong guarantee for that since a
> > broker
> > > > > could
> > > > > > >> be
> > > > > > >> > > down
> > > > > > >> > > > > for a long time. It would be better if we can have a
> > more
> > > > > > reliable
> > > > > > >> > way
> > > > > > >> > > of
> > > > > > >> > > > > dealing with deletes.
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > We propose to capture this in the "FirstDirtyOffset"
> field
> > > of
> > > > > the
> > > > > > >> > quorum
> > > > > > >> > > > record fetch response: the offset is the maximum offset
> > that
> > > > log
> > > > > > >> > > compaction
> > > > > > >> > > > has reached up to. If the follower has fetched beyond
> this
> > > > > offset
> > > > > > it
> > > > > > >> > > means
> > > > > > >> > > > itself is safe hence it has seen all records up to that
> > > > offset.
> > > > > On
> > > > > > >> > > getting
> > > > > > >> > > > the response, the follower can then decide if its end
> > offset
> > > > > > >> actually
> > > > > > >> > > below
> > > > > > >> > > > that dirty offset (and hence may miss some tombstones).
> If
> > > > > that's
> > > > > > >> the
> > > > > > >> > > case:
> > > > > > >> > > >
> > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log from the
> > very
> > > > > > >> beginning
> > > > > > >> > to
> > > > > > >> > > > catch up.
> > > > > > >> > > > 2) During that time, it would refrain itself from
> > answering
> > > > > > >> > > MetadataRequest
> > > > > > >> > > > from any clients.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> > > addresses
> > > > > the
> > > > > > >> > issue.
> > > > > > >> > > Currently, the deletion tombstone is not removed
> immediately
> > > > > after a
> > > > > > >> > round
> > > > > > >> > > of cleaning. It's removed after a delay in a subsequent
> > round
> > > of
> > > > > > >> > cleaning.
> > > > > > >> > > Consider an example where a key insertion is at offset 200
> > > and a
> > > > > > >> deletion
> > > > > > >> > > tombstone of the key is at 400. Initially,
> FirstDirtyOffset
> > is
> > > > at
> > > > > > >> 300. A
> > > > > > >> > > follower/observer fetches from offset 0  and fetches the
> key
> > > at
> > > > > > offset
> > > > > > >> > 200.
> > > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at
> 500
> > > and
> > > > > the
> > > > > > >> > > tombstone at 400 is physically removed. The
> > follower/observer
> > > > > > >> continues
> > > > > > >> > the
> > > > > > >> > > fetch, but misses offset 400. It catches all the way to
> > > > > > >> FirstDirtyOffset
> > > > > > >> > > and declares its metadata as ready. However, its metadata
> > > could
> > > > be
> > > > > > >> stale
> > > > > > >> > > since it actually misses the deletion of the key.
> > > > > > >> > >
> > > > > > >> > > Yeah good question, I should have put more details in my
> > > > > explanation
> > > > > > >> :)
> > > > > > >> >
> > > > > > >> > The idea is that we will adjust the log compaction for this
> > raft
> > > > > based
> > > > > > >> > metadata log: before more details to be explained, since we
> > have
> > > > two
> > > > > > >> types
> > > > > > >> > of "watermarks" here, whereas in Kafka the watermark
> indicates
> > > > where
> > > > > > >> every
> > > > > > >> > replica have replicated up to and in Raft the watermark
> > > indicates
> > > > > > where
> > > > > > >> the
> > > > > > >> > majority of replicas (here only indicating voters of the
> > quorum,
> > > > not
> > > > > > >> > counting observers) have replicated up to, let's call them
> > Kafka
> > > > > > >> watermark
> > > > > > >> > and Raft watermark. For this special log, we would maintain
> > both
> > > > > > >> > watermarks.
> > > > > > >> >
> > > > > > >> > When log compacting on the leader, we would only compact up
> to
> > > the
> > > > > > Kafka
> > > > > > >> > watermark, i.e. if there is at least one voter who have not
> > > > > replicated
> > > > > > >> an
> > > > > > >> > entry, it would not be compacted. The "dirty-offset" is the
> > > offset
> > > > > > that
> > > > > > >> > we've compacted up to and is communicated to other voters,
> and
> > > the
> > > > > > other
> > > > > > >> > voters would also compact up to this value --- i.e. the
> > > difference
> > > > > > here
> > > > > > >> is
> > > > > > >> > that instead of letting each replica doing log compaction
> > > > > > independently,
> > > > > > >> > we'll have the leader to decide upon which offset to compact
> > to,
> > > > and
> > > > > > >> > propagate this value to others to follow, in a more
> > coordinated
> > > > > > manner.
> > > > > > >> > Also note when there are new voters joining the quorum who
> has
> > > not
> > > > > > >> > replicated up to the dirty-offset, of because of other
> issues
> > > they
> > > > > > >> > truncated their logs to below the dirty-offset, they'd have
> to
> > > > > > >> re-bootstrap
> > > > > > >> > from the beginning, and during this period of time the
> leader
> > > > > learned
> > > > > > >> about
> > > > > > >> > this lagging voter would not advance the watermark (also it
> > > would
> > > > > not
> > > > > > >> > decrement it), and hence not compacting either, until the
> > > voter(s)
> > > > > has
> > > > > > >> > caught up to that dirty-offset.
> > > > > > >> >
> > > > > > >> > So back to your example above, before the bootstrap voter
> gets
> > > to
> > > > > 300
> > > > > > no
> > > > > > >> > log compaction would happen on the leader; and until later
> > when
> > > > the
> > > > > > >> voter
> > > > > > >> > have got to beyond 400 and hence replicated that tombstone,
> > the
> > > > log
> > > > > > >> > compaction would possibly get to that tombstone and remove
> it.
> > > Say
> > > > > > >> later it
> > > > > > >> > the leader's log compaction reaches 500, it can send this
> back
> > > to
> > > > > the
> > > > > > >> voter
> > > > > > >> > who can then also compact locally up to 500.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we need
> > the
> > > > > epoch
> > > > > > >> > > > > corresponding to VotedId? Over time, the same broker
> Id
> > > > could
> > > > > be
> > > > > > >> > voted
> > > > > > >> > > in
> > > > > > >> > > > > different generations with different epoch.
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > > "LeaderEpoch"
> > > > > > >> field
> > > > > > >> > in
> > > > > > >> > > > that file is corresponding to the "latest known leader
> > > epoch",
> > > > > not
> > > > > > >> the
> > > > > > >> > > > "current leader epoch". For example, if the current
> epoch
> > is
> > > > N,
> > > > > > and
> > > > > > >> > then
> > > > > > >> > > a
> > > > > > >> > > > vote-request with epoch N+1 is received and the voter
> > > granted
> > > > > the
> > > > > > >> vote
> > > > > > >> > > for
> > > > > > >> > > > it, then it means for this voter it knows the "latest
> > epoch"
> > > > is
> > > > > N
> > > > > > +
> > > > > > >> 1
> > > > > > >> > > > although it is unknown if that sending candidate will
> > indeed
> > > > > > become
> > > > > > >> the
> > > > > > >> > > new
> > > > > > >> > > > leader (which would only be notified via begin-quorum
> > > > request).
> > > > > > >> > However,
> > > > > > >> > > > when persisting the quorum state, we would encode
> > > leader-epoch
> > > > > to
> > > > > > >> N+1,
> > > > > > >> > > > while the leaderId to be the older leader.
> > > > > > >> > > >
> > > > > > >> > > > But now thinking about this a bit more, I feel we should
> > use
> > > > two
> > > > > > >> > separate
> > > > > > >> > > > epochs, one for the "lates known" and one for the
> > "current"
> > > to
> > > > > > pair
> > > > > > >> > with
> > > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch before
> > the
> > > > new
> > > > > > >> leader
> > > > > > >> > > is actually elected, right.
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > FetchQuorumRecords
> > > > API
> > > > > to
> > > > > > >> > > indicate
> > > > > > >> > > > > that the follower has fetched from an invalid offset
> and
> > > > > should
> > > > > > >> > > truncate
> > > > > > >> > > > to
> > > > > > >> > > > > the offset/epoch indicated in the response." Observers
> > > can't
> > > > > > >> truncate
> > > > > > >> > > > their
> > > > > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > I'm not sure if I understand your question? Observers
> > should
> > > > > still
> > > > > > >> be
> > > > > > >> > > able
> > > > > > >> > > > to truncate their logs as well.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > Hmm, I thought only the quorum nodes have local logs and
> > > > observers
> > > > > > >> don't?
> > > > > > >> > >
> > > > > > >> > > > 107. "The leader will continue sending BeginQuorumEpoch
> to
> > > > each
> > > > > > >> known
> > > > > > >> > > > voter
> > > > > > >> > > > > until it has received its endorsement." If a voter is
> > down
> > > > > for a
> > > > > > >> long
> > > > > > >> > > > time,
> > > > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> > > overhead.
> > > > > > >> > Similarly,
> > > > > > >> > > > if a
> > > > > > >> > > > > follower stops sending FetchQuorumRecords, does the
> > leader
> > > > > keep
> > > > > > >> > sending
> > > > > > >> > > > > BeginQuorumEpoch?
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > > > > >> > begin-quorum-epoch
> > > > > > >> > > > request is for voters to quickly get the new leader
> > > > information;
> > > > > > >> > however
> > > > > > >> > > > even if they do not get them they can still eventually
> > learn
> > > > > about
> > > > > > >> that
> > > > > > >> > > > from others via gossiping FindQuorum. I think we can
> > adjust
> > > > the
> > > > > > >> logic
> > > > > > >> > to
> > > > > > >> > > > e.g. exponential back-off or with a limited num.retries.
> > > > > > >> > > >
> > > > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > > > >> FetchQuorumRecords
> > > > > > >> > > > already, it means that follower already knows that the
> > > broker
> > > > is
> > > > > > the
> > > > > > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch;
> > > > however
> > > > > > it
> > > > > > >> is
> > > > > > >> > > > possible that after a follower sends FetchQuorumRecords
> > > > already,
> > > > > > >> > suddenly
> > > > > > >> > > > it stops send it (possibly because it learned about a
> > higher
> > > > > epoch
> > > > > > >> > > leader),
> > > > > > >> > > > and hence this broker may be a "zombie" leader and we
> > > propose
> > > > to
> > > > > > use
> > > > > > >> > the
> > > > > > >> > > > fetch.timeout to let the leader to try to verify if it
> has
> > > > > already
> > > > > > >> been
> > > > > > >> > > > stale.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > It just seems that we should handle these two cases in a
> > > > > consistent
> > > > > > >> way?
> > > > > > >> > >
> > > > > > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords
> > > from a
> > > > > > >> follower
> > > > > > >> > could mean that we no longer needs to send BeginQuorumEpoch
> > > > anymore
> > > > > > ---
> > > > > > >> and
> > > > > > >> > it is already part of our current implementations in
> > > > > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > Thanks,
> > > > > > >> > >
> > > > > > >> > > Jun
> > > > > > >> > >
> > > > > > >> > > >
> > > > > > >> > > > >
> > > > > > >> > > > > Jun
> > > > > > >> > > > >
> > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > >> >
> > > > > > >> > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hello Leonard,
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks for your comments, I'm relying in line below:
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > > > > > >> > wge@confluent.io>
> > > > > > >> > > > > > wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Hi Kafka developers,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > It's great to see this proposal and it took me
> some
> > > time
> > > > > to
> > > > > > >> > finish
> > > > > > >> > > > > > reading
> > > > > > >> > > > > > > it.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > And I have the following questions about the
> > Proposal:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >    - How do we plan to test this design to ensure
> > its
> > > > > > >> > correctness?
> > > > > > >> > > Or
> > > > > > >> > > > > > more
> > > > > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’
> > based
> > > > > model
> > > > > > >> is
> > > > > > >> > > > > > functional
> > > > > > >> > > > > > > and
> > > > > > >> > > > > > >    correct given that it is different from the
> > > original
> > > > > RAFT
> > > > > > >> > > > > > implementation
> > > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > We have two planned verifications on the correctness
> > and
> > > > > > >> liveness
> > > > > > >> > of
> > > > > > >> > > > the
> > > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > > > > > >> > > > > >
> > > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > > >> > > > > >
> > > > > > >> > > > > >    - Have we considered any sensible defaults for
> the
> > > > > > >> > configuration,
> > > > > > >> > > > i.e.
> > > > > > >> > > > > > >    all the election timeout, fetch time out, etc.?
> > Or
> > > we
> > > > > > want
> > > > > > >> to
> > > > > > >> > > > leave
> > > > > > >> > > > > > > this to
> > > > > > >> > > > > > >    a later stage when we do the performance
> testing,
> > > > etc.
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > This is a good question, the reason we did not set
> any
> > > > > default
> > > > > > >> > values
> > > > > > >> > > > for
> > > > > > >> > > > > > the timeout configurations is that we think it may
> > take
> > > > some
> > > > > > >> > > > benchmarking
> > > > > > >> > > > > > experiments to get these defaults right. Some
> > high-level
> > > > > > >> principles
> > > > > > >> > > to
> > > > > > >> > > > > > consider: 1) the fetch.timeout should be around the
> > same
> > > > > scale
> > > > > > >> with
> > > > > > >> > > zk
> > > > > > >> > > > > > session timeout, which is now 18 seconds by default
> --
> > > in
> > > > > > >> practice
> > > > > > >> > > > we've
> > > > > > >> > > > > > seen unstable networks having more than 10 secs of
> > > > transient
> > > > > > >> > > > > connectivity,
> > > > > > >> > > > > > 2) the election.timeout, however, should be smaller
> > than
> > > > the
> > > > > > >> fetch
> > > > > > >> > > > > timeout
> > > > > > >> > > > > > as is also suggested as a practical optimization in
> > > > > > literature:
> > > > > > >> > > > > >
> > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > > >> > > > > >
> > > > > > >> > > > > > Some more discussions can be found here:
> > > > > > >> > > > > >
> > > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - Have we considered piggybacking
> > > `BeginQuorumEpoch`
> > > > > with
> > > > > > >> the
> > > > > > >> > `
> > > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing
> something
> > > > > obvious
> > > > > > >> but
> > > > > > >> > I
> > > > > > >> > > am
> > > > > > >> > > > > > just
> > > > > > >> > > > > > >    wondering why don’t we just use the
> `FindQuorum`
> > > and
> > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader
> ->
> > > > other
> > > > > > >> voter
> > > > > > >> > > > > > followers, while FindQuorum / Fetch are sent from
> > > follower
> > > > > to
> > > > > > >> > leader.
> > > > > > >> > > > > > Arguably one can eventually realize the new leader
> and
> > > > epoch
> > > > > > via
> > > > > > >> > > > > gossiping
> > > > > > >> > > > > > FindQuorum, but that could in practice require a
> long
> > > > delay.
> > > > > > >> > Having a
> > > > > > >> > > > > > leader -> other voters request helps the new leader
> > > epoch
> > > > to
> > > > > > be
> > > > > > >> > > > > propagated
> > > > > > >> > > > > > faster under a pull model.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - And about the `FetchQuorumRecords` response
> > > schema,
> > > > > in
> > > > > > >> the
> > > > > > >> > > > > `Records`
> > > > > > >> > > > > > >    field of the response, is it just one record or
> > all
> > > > the
> > > > > > >> > records
> > > > > > >> > > > > > starting
> > > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> > efficient
> > > > if
> > > > > we
> > > > > > >> sent
> > > > > > >> > > all
> > > > > > >> > > > > the
> > > > > > >> > > > > > >    records during the bootstrapping of the
> brokers.
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> > > > > starting
> > > > > > >> > offset
> > > > > > >> > > of
> > > > > > >> > > > > the
> > > > > > >> > > > > > batch of records.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - Regarding the disruptive broker issues, does
> > our
> > > > pull
> > > > > > >> based
> > > > > > >> > > > model
> > > > > > >> > > > > > >    suffer from it? If so, have we considered the
> > > > Pre-Vote
> > > > > > >> stage?
> > > > > > >> > If
> > > > > > >> > > > > not,
> > > > > > >> > > > > > > why?
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > The disruptive broker is stated in the original Raft
> > > paper
> > > > > > >> which is
> > > > > > >> > > the
> > > > > > >> > > > > > result of the push model design. Our analysis showed
> > > that
> > > > > with
> > > > > > >> the
> > > > > > >> > > pull
> > > > > > >> > > > > > model it is no longer an issue.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Thanks a lot for putting this up, and I hope that
> my
> > > > > > questions
> > > > > > >> > can
> > > > > > >> > > be
> > > > > > >> > > > > of
> > > > > > >> > > > > > > some value to make this KIP better.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Hope to hear from you soon!
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Best wishes,
> > > > > > >> > > > > > > Leonard
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > > > >> cmccabe@apache.org
> > > > > > >> > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > Hi Jason,
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > I haven't had a chance to read in detail, but I
> > read
> > > > the
> > > > > > >> > outline
> > > > > > >> > > > and
> > > > > > >> > > > > a
> > > > > > >> > > > > > > few
> > > > > > >> > > > > > > > things jumped out at me.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > First, for every epoch that is 32 bits rather
> than
> > > > 64, I
> > > > > > >> sort
> > > > > > >> > of
> > > > > > >> > > > > wonder
> > > > > > >> > > > > > > if
> > > > > > >> > > > > > > > that's a good long-term choice.  I keep reading
> > > about
> > > > > > stuff
> > > > > > >> > like
> > > > > > >> > > > > this:
> > > > > > >> > > > > > > >
> > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > .
> > > > > > >> > > Obviously,
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > > JIRA is about zxid, which increments much faster
> > > than
> > > > we
> > > > > > >> expect
> > > > > > >> > > > these
> > > > > > >> > > > > > > > leader epochs to, but it would still be good to
> > see
> > > > some
> > > > > > >> rough
> > > > > > >> > > > > > > calculations
> > > > > > >> > > > > > > > about how long 32 bits (or really, 31 bits) will
> > > last
> > > > us
> > > > > > in
> > > > > > >> the
> > > > > > >> > > > cases
> > > > > > >> > > > > > > where
> > > > > > >> > > > > > > > we're using it, and what the space savings we're
> > > > getting
> > > > > > >> really
> > > > > > >> > > is.
> > > > > > >> > > > > It
> > > > > > >> > > > > > > > seems like in most cases the tradeoff may not be
> > > worth
> > > > > it?
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Another thing I've been thinking about is how we
> > do
> > > > > > >> > > > bootstrapping.  I
> > > > > > >> > > > > > > > would prefer to be in a world where formatting a
> > new
> > > > > Kafka
> > > > > > >> node
> > > > > > >> > > > was a
> > > > > > >> > > > > > > first
> > > > > > >> > > > > > > > class operation explicitly initiated by the
> admin,
> > > > > rather
> > > > > > >> than
> > > > > > >> > > > > > something
> > > > > > >> > > > > > > > that happened implicitly when you started up the
> > > > broker
> > > > > > and
> > > > > > >> > > things
> > > > > > >> > > > > > > "looked
> > > > > > >> > > > > > > > blank."
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > The first problem is that things can "look
> blank"
> > > > > > >> accidentally
> > > > > > >> > if
> > > > > > >> > > > the
> > > > > > >> > > > > > > > storage system is having a bad day.  Clearly in
> > the
> > > > > > non-Raft
> > > > > > >> > > world,
> > > > > > >> > > > > > this
> > > > > > >> > > > > > > > leads to data loss if the broker that is
> > (re)started
> > > > > this
> > > > > > >> way
> > > > > > >> > was
> > > > > > >> > > > the
> > > > > > >> > > > > > > > leader for some partitions.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > The second problem is that we have a bit of a
> > > chicken
> > > > > and
> > > > > > >> egg
> > > > > > >> > > > problem
> > > > > > >> > > > > > > with
> > > > > > >> > > > > > > > certain configuration keys.  For example, maybe
> > you
> > > > want
> > > > > > to
> > > > > > >> > > > configure
> > > > > > >> > > > > > > some
> > > > > > >> > > > > > > > connection security settings in your cluster,
> but
> > > you
> > > > > > don't
> > > > > > >> > want
> > > > > > >> > > > them
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > ever be stored in a plaintext config file.  (For
> > > > > example,
> > > > > > >> SCRAM
> > > > > > >> > > > > > > passwords,
> > > > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > > > configuration,
> > > > > > >> but
> > > > > > >> > > > that
> > > > > > >> > > > > > > brings
> > > > > > >> > > > > > > > up the chicken and egg problem.  The broker
> needs
> > to
> > > > be
> > > > > > >> > > configured
> > > > > > >> > > > to
> > > > > > >> > > > > > > know
> > > > > > >> > > > > > > > how to talk to you, but you need to configure it
> > > > before
> > > > > > you
> > > > > > >> can
> > > > > > >> > > > talk
> > > > > > >> > > > > to
> > > > > > >> > > > > > > > it.  Using an external secret manager like Vault
> > is
> > > > one
> > > > > > way
> > > > > > >> to
> > > > > > >> > > > solve
> > > > > > >> > > > > > > this,
> > > > > > >> > > > > > > > but not everyone uses an external secret
> manager.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > quorum.voters seems like a similar configuration
> > > key.
> > > > > In
> > > > > > >> the
> > > > > > >> > > > current
> > > > > > >> > > > > > > KIP,
> > > > > > >> > > > > > > > this is only read if there is no other
> > configuration
> > > > > > >> specifying
> > > > > > >> > > the
> > > > > > >> > > > > > > quorum
> > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> > > > wouldn't
> > > > > > need
> > > > > > >> > this
> > > > > > >> > > > key
> > > > > > >> > > > > > > > because we could assume that there was always
> > quorum
> > > > > > >> > information
> > > > > > >> > > > > stored
> > > > > > >> > > > > > > > locally.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > best,
> > > > > > >> > > > > > > > Colin
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson
> > > wrote:
> > > > > > >> > > > > > > > > Hi All,
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > >> > > > > > > > .
> > > > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > > > ultimately
> > > > > > >> replace
> > > > > > >> > > > > > Zookeeper
> > > > > > >> > > > > > > > > as
> > > > > > >> > > > > > > > > documented in KIP-500. Please take a look and
> > > share
> > > > > your
> > > > > > >> > > > thoughts.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > A few minor notes to set the stage a little
> bit:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > - This KIP does not specify the structure of
> the
> > > > > > messages
> > > > > > >> > used
> > > > > > >> > > to
> > > > > > >> > > > > > > > represent
> > > > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> > > internal
> > > > > API
> > > > > > >> that
> > > > > > >> > > will
> > > > > > >> > > > > be
> > > > > > >> > > > > > > used
> > > > > > >> > > > > > > > > by the controller. Expect these to come in
> later
> > > > > > >> proposals.
> > > > > > >> > > Here
> > > > > > >> > > > we
> > > > > > >> > > > > > are
> > > > > > >> > > > > > > > > primarily concerned with the replication
> > protocol
> > > > and
> > > > > > >> basic
> > > > > > >> > > > > > operational
> > > > > > >> > > > > > > > > mechanics.
> > > > > > >> > > > > > > > > - We expect many details to change as we get
> > > closer
> > > > to
> > > > > > >> > > > integration
> > > > > > >> > > > > > with
> > > > > > >> > > > > > > > > the controller. Any changes we make will be
> made
> > > > > either
> > > > > > as
> > > > > > >> > > > > amendments
> > > > > > >> > > > > > > to
> > > > > > >> > > > > > > > > this KIP or, in the case of larger changes, as
> > new
> > > > > > >> proposals.
> > > > > > >> > > > > > > > > - We have a prototype implementation which I
> > will
> > > > put
> > > > > > >> online
> > > > > > >> > > > within
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > > next week which may help in understanding some
> > > > > details.
> > > > > > It
> > > > > > >> > has
> > > > > > >> > > > > > > diverged a
> > > > > > >> > > > > > > > > little bit from our proposal, so I am taking a
> > > > little
> > > > > > >> time to
> > > > > > >> > > > bring
> > > > > > >> > > > > > it
> > > > > > >> > > > > > > in
> > > > > > >> > > > > > > > > line. I'll post an update to this thread when
> it
> > > is
> > > > > > >> available
> > > > > > >> > > for
> > > > > > >> > > > > > > review.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Finally, I want to mention that this proposal
> > was
> > > > > > drafted
> > > > > > >> by
> > > > > > >> > > > > myself,
> > > > > > >> > > > > > > > Boyang
> > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Thanks,
> > > > > > >> > > > > > > > > Jason
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > --
> > > > > > >> > > > > > > Leonard Ge
> > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > --
> > > > > > >> > > > > > -- Guozhang
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > --
> > > > > > >> > > > -- Guozhang
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > --
> > > > > > >> > -- Guozhang
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Tom Bentley <tb...@redhat.com>.
Hi Jason, Ghouzang and Boyang,

First, thanks for the KIP. I've not a few minor suggestions and nits, but
on the whole it was pretty clear and understandable.

1. § Motivation

    "We have intentionally avoided any assumption about the representation
of the log and its semantics."

I found this a bit confusing because there's a whole section about the
assumed log structure later on. I think something like:

    "We make no assumption about the physical log format and minimal
assumptions about it's logical structure (see §Log Structure)."

might make things a little clearer, though that suggestion does omit any
claim about semantics.

Furthermore, in the "Log Structure" section it ends with "Kafka's current
v2 message format version supports everything we need, so we will assume
that."
All you need to say here is that "the v2 message format has the
assumed/required properties", saying you're assuming the v2 message format
negates the rest of the paragraph.


2. § Configurations

The format for voters in the `quorum.voters` is potentially a little
confusing because the separation of the broker id from the broker host name
with a dot means the whole thing is syntactically indistinguishable from a
host name. Using a different separator would avoid such ambiguity.


3. § Leader Election and Data Replication

    "The key functionalities of any consensus protocol are leader election
and data replication. The protocol for these two functionalities consists
of 5 core RPCs:"

This sentence is followed by a list of 4 (rather than 5) items. I don't
think you meant to imply DescribeQuorum is a core RPC.


4. § BeginQuorumEpoch

    "observers will discover the new leader through either
the DiscoverBrokers or Fetch APIs"

DiscoverBrokers isn't defined in this KIP (I think it was when I read an
earlier version), and I don't recall seeing another KIP which describes it.


5. § EndQuorumEpoch

    "If the node's priority is highest, it will become candidate
immediately instead of waiting for next poll."

This is clarified in the later section on EndquorumEpoch request handling,
but I found the wording " instead of waiting for next poll" to be
confusing. Maybe "instead of waiting for next election timeout" would be
clearer?


6. § Fetch

The diagram from the Raft dissertation is a broken link (I'm guessing for
anyone lacking access to https://confluentinc.atlassian.net).


7. § Fetch

    "This proposal also extends the Fetch response to include a separate
field for the receiver of the request to indicate the current leader."

I think "receiver of the request" should be "receiver of the response",
shouldn't it?


8. § Fetch § Response Schema

The `NextOffsetAndEpoch` and `CurrentLeader` field in the fetch response
lack `about` properties.


9. § FetchRequest Handling

Item 1.a. is covering only the case where the fetcher is a voter (thus in
receipt of BeginQuorumEpoch), it doesn't cover observer fetchers.

Also this algorithm talks about "the leader epoch" being larger or smaller,
but it's not immediately clear whether this is the leader's epoch or the
leader epoch in the fetch request and which is the epoch being compared
against. It makes the comparisons harder to understand if you have to guess
the order of the operands.


10. § Discussion: Replication Progress Timeout for Zombie Leader

    "Note that the node will remain a leader until it finds that it has
been supplanted by another voter."

I don't quite follow this: Why is a _voter_ involved here? Surely it's
simply a matter of the leader observing that a new leader exists which
determines whether it steps down.


11. § DescribeQuorum

This seems to be defined also in KIP-642.


12. § Tooling support

`kafka-metadata-quorum.sh`: Given that the protocol is essentially
independent of the partitions for which the quorum is established
it seems a little odd to name the tool so that it can't later be extended
to support quorums for other partitions.
For example why not `kafka-quorum.sh --metadata --describe` and later
support `kafka-quorum.sh --topic-partitions foo-0,foo-1 --describe`.
Obviously you don't need `--metadata` (it could be the default if
`--topic-partitions` was omitted), it's more a point about the name of the
tool.

Again in this section, on the one hand the list says `--describe` supports
two arguments (status and replication).
On the other hand the example shows a plain, argumentless `--describe`.


13. § Metrics

i. I wonder about including `raft` in the metric names, since it's not
really exposed elsewhere in the public API.

ii. Similarly to the tool/script, we end up with metrics which are
implicitly about a partition. If we use quorums for other partitions in the
future we'll need to replicate a lot of these metrics, which would make it
harder to abstract across collections of metrics.


Thanks again for the KIP,

Tom



On Mon, Jul 13, 2020 at 5:50 PM Jason Gustafson <ja...@confluent.io> wrote:

> Hi All,
>
> Just a quick update on the proposal. We have decided to move quorum
> reassignment to a separate KIP:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> .
> The way this ties into cluster bootstrapping is complicated, so we felt we
> needed a bit more time for validation. That leaves the core of this
> proposal as quorum-based replication. If there are no further comments, we
> will plan to start a vote later this week.
>
> Thanks,
> Jason
>
> On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com> wrote:
>
> > @Jun Rao <ju...@gmail.com>
> >
> > Regarding your comment about log compaction. After some deep-diving into
> > this we've decided to propose a new snapshot-based log cleaning mechanism
> > which would be used to replace the current compaction mechanism for this
> > meta log. A new KIP will be proposed specifically for this idea.
> >
> > All,
> >
> > I've updated the KIP wiki a bit updating one config "
> > election.jitter.max.ms"
> > to "election.backoff.max.ms" to make it more clear about the usage: the
> > configured value will be the upper bound of the binary exponential
> backoff
> > time after a failed election, before starting a new one.
> >
> >
> >
> > Guozhang
> >
> >
> >
> > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> > > Thanks for the suggestions Guozhang.
> > >
> > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hello Boyang,
> > > >
> > > > Thanks for the updated information. A few questions here:
> > > >
> > > > 1) Should the quorum-file also update to support multi-raft?
> > > >
> > > > I'm neutral about this, as we don't know yet how the multi-raft
> modules
> > > would behave. If
> > > we have different threads operating different raft groups,
> consolidating
> > > the `checkpoint` files seems
> > > not reasonable. We could always add `multi-quorum-file` later if
> > possible.
> > >
> > > 2) In the previous proposal, there's fields in the FetchQuorumRecords
> > like
> > > > latestDirtyOffset, is that dropped intentionally?
> > > >
> > > > I dropped the latestDirtyOffset since it is associated with the log
> > > compaction discussion. This is beyond this KIP scope and we could
> > > potentially get a separate KIP to talk about it.
> > >
> > >
> > > > 3) I think we also need to elaborate a bit more details regarding
> when
> > to
> > > > send metadata request and discover-brokers; currently we only
> discussed
> > > > during bootstrap how these requests would be sent. I think the
> > following
> > > > scenarios would also need these requests
> > > >
> > > > 3.a) As long as a broker does not know the current quorum (including
> > the
> > > > leader and the voters), it should continue periodically ask other
> > brokers
> > > > via "metadata.
> > > > 3.b) As long as a broker does not know all the current quorum voter's
> > > > connections, it should continue periodically ask other brokers via
> > > > "discover-brokers".
> > > > 3.c) When the leader's fetch timeout elapsed, it should send metadata
> > > > request.
> > > >
> > > > Make sense, will add to the KIP.
> > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > reluctanthero104@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey all,
> > > > >
> > > > > follow-up on the previous email, we made some more updates:
> > > > >
> > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > > > multi-raft.
> > > > >
> > > > > 2. We add observer status into the DescribeQuorumResponse as we see
> > it
> > > > is a
> > > > > low hanging fruit which is very useful for user debugging and
> > > > reassignment.
> > > > >
> > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which
> is
> > > > purely
> > > > > in charge of discovering broker connections in a gossip manner. The
> > > > quorum
> > > > > leader discovery is piggy-back on the Metadata RPC for the topic
> > > > partition
> > > > > leader, which in our case is the single metadata partition for the
> > > > version
> > > > > one.
> > > > >
> > > > > Let me know if you have any questions.
> > > > >
> > > > > Boyang
> > > > >
> > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey all,
> > > > > >
> > > > > > Thanks for the great discussions so far. I'm posting some KIP
> > updates
> > > > > from
> > > > > > our working group discussion:
> > > > > >
> > > > > > 1. We will be changing the core RPCs from single-raft API to
> > > > multi-raft.
> > > > > > This means all protocols will be "batch" in the first version,
> but
> > > the
> > > > > KIP
> > > > > > itself only illustrates the design for a single metadata topic
> > > > partition.
> > > > > > The reason is to "keep the door open" for future extensions of
> this
> > > > piece
> > > > > > of module such as a sharded controller or general quorum based
> > topic
> > > > > > replication, beyond the current Kafka replication protocol.
> > > > > >
> > > > > > 2. We will piggy-back on the current Kafka Fetch API instead of
> > > > inventing
> > > > > > a new FetchQuorumRecords RPC. The motivation is about the same as
> > #1
> > > as
> > > > > > well as making the integration work easier, instead of letting
> two
> > > > > similar
> > > > > > RPCs diverge.
> > > > > >
> > > > > > 3. In the EndQuorumEpoch protocol, instead of only sending the
> > > request
> > > > to
> > > > > > the most caught-up voter, we shall broadcast the information to
> all
> > > > > voters,
> > > > > > with a sorted voter list in descending order of their
> corresponding
> > > > > > replicated offset. In this way, the top voter will become a
> > candidate
> > > > > > immediately, while the other voters shall wait for an exponential
> > > > > back-off
> > > > > > to trigger elections, which helps ensure the top voter gets
> > elected,
> > > > and
> > > > > > the election eventually happens when the top voter is not
> > responsive.
> > > > > >
> > > > > > Please see the updated KIP and post any questions or concerns on
> > the
> > > > > > mailing thread.
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > >> Hi, Guozhang and Jason,
> > > > > >>
> > > > > >> Thanks for the reply. A couple of more replies.
> > > > > >>
> > > > > >> 102. Still not sure about this. How is the tombstone issue
> > addressed
> > > > in
> > > > > >> the
> > > > > >> non-voter and the observer.  They can die at any point and
> restart
> > > at
> > > > an
> > > > > >> arbitrary later time, and the advancing of the firstDirty offset
> > and
> > > > the
> > > > > >> removal of the tombstone can happen independently.
> > > > > >>
> > > > > >> 106. I agree that it would be less confusing if we used "epoch"
> > > > instead
> > > > > of
> > > > > >> "leader epoch" consistently.
> > > > > >>
> > > > > >> Jun
> > > > > >>
> > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> wangguoz@gmail.com>
> > > > > wrote:
> > > > > >>
> > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > >> >
> > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > >> >
> > > > > >> > > Hi, Guozhang,
> > > > > >> > >
> > > > > >> > > Thanks for the reply. A few more replies inlined below.
> > > > > >> > >
> > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > >> wrote:
> > > > > >> > >
> > > > > >> > > > Hello Jun,
> > > > > >> > > >
> > > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > > >> > > >
> > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > >> > > > > 101.1 Currently, we support auto broker id generation.
> Is
> > > this
> > > > > >> > > supported
> > > > > >> > > > > for bootstrap brokers?
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > > The vote ids would just be the broker ids.
> > "bootstrap.servers"
> > > > > >> would be
> > > > > >> > > > similar to what client configs have today, where
> > > "quorum.voters"
> > > > > >> would
> > > > > >> > be
> > > > > >> > > > pre-defined config values.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > My question was on the auto generated broker id. Currently,
> > the
> > > > > broker
> > > > > >> > can
> > > > > >> > > choose to have its broker Id auto generated. The generation
> is
> > > > done
> > > > > >> > through
> > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how
> the
> > > > > broker
> > > > > >> id
> > > > > >> > is
> > > > > >> > > auto generated. "quorum.voters" also can't be set statically
> > if
> > > > > broker
> > > > > >> > ids
> > > > > >> > > are auto generated.
> > > > > >> > >
> > > > > >> > > Jason has explained some ideas that we've discussed so far,
> > the
> > > > > >> reason we
> > > > > >> > intentional did not include them so far is that we feel it is
> > > > out-side
> > > > > >> the
> > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> > > definitely
> > > > > >> > address them though.
> > > > > >> >
> > > > > >> > On the high-level, our belief is that "joining a quorum" and
> > > > "joining
> > > > > >> (or
> > > > > >> > more specifically, registering brokers in) the cluster" would
> be
> > > > > >> > de-coupled a bit, where the former should be completed before
> we
> > > do
> > > > > the
> > > > > >> > latter. More specifically, assuming the quorum is already up
> and
> > > > > >> running,
> > > > > >> > after the newly started broker found the leader of the quorum
> it
> > > can
> > > > > >> send a
> > > > > >> > specific RegisterBroker request including its listener /
> > protocol
> > > /
> > > > > etc,
> > > > > >> > and upon handling it the leader can send back the uniquely
> > > generated
> > > > > >> broker
> > > > > >> > id to the new broker, while also executing the
> "startNewBroker"
> > > > > >> callback as
> > > > > >> > the controller.
> > > > > >> >
> > > > > >> >
> > > > > >> > >
> > > > > >> > > > > 102. Log compaction. One weak spot of log compaction is
> > for
> > > > the
> > > > > >> > > consumer
> > > > > >> > > > to
> > > > > >> > > > > deal with deletes. When a key is deleted, it's retained
> > as a
> > > > > >> > tombstone
> > > > > >> > > > > first and then physically removed. If a client misses
> the
> > > > > >> tombstone
> > > > > >> > > > > (because it's physically removed), it may not be able to
> > > > update
> > > > > >> its
> > > > > >> > > > > metadata properly. The way we solve this in Kafka is
> based
> > > on
> > > > a
> > > > > >> > > > > configuration (log.cleaner.delete.retention.ms) and we
> > > > expect a
> > > > > >> > > consumer
> > > > > >> > > > > having seen an old key to finish reading the deletion
> > > > tombstone
> > > > > >> > within
> > > > > >> > > > that
> > > > > >> > > > > time. There is no strong guarantee for that since a
> broker
> > > > could
> > > > > >> be
> > > > > >> > > down
> > > > > >> > > > > for a long time. It would be better if we can have a
> more
> > > > > reliable
> > > > > >> > way
> > > > > >> > > of
> > > > > >> > > > > dealing with deletes.
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > > We propose to capture this in the "FirstDirtyOffset" field
> > of
> > > > the
> > > > > >> > quorum
> > > > > >> > > > record fetch response: the offset is the maximum offset
> that
> > > log
> > > > > >> > > compaction
> > > > > >> > > > has reached up to. If the follower has fetched beyond this
> > > > offset
> > > > > it
> > > > > >> > > means
> > > > > >> > > > itself is safe hence it has seen all records up to that
> > > offset.
> > > > On
> > > > > >> > > getting
> > > > > >> > > > the response, the follower can then decide if its end
> offset
> > > > > >> actually
> > > > > >> > > below
> > > > > >> > > > that dirty offset (and hence may miss some tombstones). If
> > > > that's
> > > > > >> the
> > > > > >> > > case:
> > > > > >> > > >
> > > > > >> > > > 1) Naively, it could re-bootstrap metadata log from the
> very
> > > > > >> beginning
> > > > > >> > to
> > > > > >> > > > catch up.
> > > > > >> > > > 2) During that time, it would refrain itself from
> answering
> > > > > >> > > MetadataRequest
> > > > > >> > > > from any clients.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> > addresses
> > > > the
> > > > > >> > issue.
> > > > > >> > > Currently, the deletion tombstone is not removed immediately
> > > > after a
> > > > > >> > round
> > > > > >> > > of cleaning. It's removed after a delay in a subsequent
> round
> > of
> > > > > >> > cleaning.
> > > > > >> > > Consider an example where a key insertion is at offset 200
> > and a
> > > > > >> deletion
> > > > > >> > > tombstone of the key is at 400. Initially, FirstDirtyOffset
> is
> > > at
> > > > > >> 300. A
> > > > > >> > > follower/observer fetches from offset 0  and fetches the key
> > at
> > > > > offset
> > > > > >> > 200.
> > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500
> > and
> > > > the
> > > > > >> > > tombstone at 400 is physically removed. The
> follower/observer
> > > > > >> continues
> > > > > >> > the
> > > > > >> > > fetch, but misses offset 400. It catches all the way to
> > > > > >> FirstDirtyOffset
> > > > > >> > > and declares its metadata as ready. However, its metadata
> > could
> > > be
> > > > > >> stale
> > > > > >> > > since it actually misses the deletion of the key.
> > > > > >> > >
> > > > > >> > > Yeah good question, I should have put more details in my
> > > > explanation
> > > > > >> :)
> > > > > >> >
> > > > > >> > The idea is that we will adjust the log compaction for this
> raft
> > > > based
> > > > > >> > metadata log: before more details to be explained, since we
> have
> > > two
> > > > > >> types
> > > > > >> > of "watermarks" here, whereas in Kafka the watermark indicates
> > > where
> > > > > >> every
> > > > > >> > replica have replicated up to and in Raft the watermark
> > indicates
> > > > > where
> > > > > >> the
> > > > > >> > majority of replicas (here only indicating voters of the
> quorum,
> > > not
> > > > > >> > counting observers) have replicated up to, let's call them
> Kafka
> > > > > >> watermark
> > > > > >> > and Raft watermark. For this special log, we would maintain
> both
> > > > > >> > watermarks.
> > > > > >> >
> > > > > >> > When log compacting on the leader, we would only compact up to
> > the
> > > > > Kafka
> > > > > >> > watermark, i.e. if there is at least one voter who have not
> > > > replicated
> > > > > >> an
> > > > > >> > entry, it would not be compacted. The "dirty-offset" is the
> > offset
> > > > > that
> > > > > >> > we've compacted up to and is communicated to other voters, and
> > the
> > > > > other
> > > > > >> > voters would also compact up to this value --- i.e. the
> > difference
> > > > > here
> > > > > >> is
> > > > > >> > that instead of letting each replica doing log compaction
> > > > > independently,
> > > > > >> > we'll have the leader to decide upon which offset to compact
> to,
> > > and
> > > > > >> > propagate this value to others to follow, in a more
> coordinated
> > > > > manner.
> > > > > >> > Also note when there are new voters joining the quorum who has
> > not
> > > > > >> > replicated up to the dirty-offset, of because of other issues
> > they
> > > > > >> > truncated their logs to below the dirty-offset, they'd have to
> > > > > >> re-bootstrap
> > > > > >> > from the beginning, and during this period of time the leader
> > > > learned
> > > > > >> about
> > > > > >> > this lagging voter would not advance the watermark (also it
> > would
> > > > not
> > > > > >> > decrement it), and hence not compacting either, until the
> > voter(s)
> > > > has
> > > > > >> > caught up to that dirty-offset.
> > > > > >> >
> > > > > >> > So back to your example above, before the bootstrap voter gets
> > to
> > > > 300
> > > > > no
> > > > > >> > log compaction would happen on the leader; and until later
> when
> > > the
> > > > > >> voter
> > > > > >> > have got to beyond 400 and hence replicated that tombstone,
> the
> > > log
> > > > > >> > compaction would possibly get to that tombstone and remove it.
> > Say
> > > > > >> later it
> > > > > >> > the leader's log compaction reaches 500, it can send this back
> > to
> > > > the
> > > > > >> voter
> > > > > >> > who can then also compact locally up to 500.
> > > > > >> >
> > > > > >> >
> > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we need
> the
> > > > epoch
> > > > > >> > > > > corresponding to VotedId? Over time, the same broker Id
> > > could
> > > > be
> > > > > >> > voted
> > > > > >> > > in
> > > > > >> > > > > different generations with different epoch.
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > "LeaderEpoch"
> > > > > >> field
> > > > > >> > in
> > > > > >> > > > that file is corresponding to the "latest known leader
> > epoch",
> > > > not
> > > > > >> the
> > > > > >> > > > "current leader epoch". For example, if the current epoch
> is
> > > N,
> > > > > and
> > > > > >> > then
> > > > > >> > > a
> > > > > >> > > > vote-request with epoch N+1 is received and the voter
> > granted
> > > > the
> > > > > >> vote
> > > > > >> > > for
> > > > > >> > > > it, then it means for this voter it knows the "latest
> epoch"
> > > is
> > > > N
> > > > > +
> > > > > >> 1
> > > > > >> > > > although it is unknown if that sending candidate will
> indeed
> > > > > become
> > > > > >> the
> > > > > >> > > new
> > > > > >> > > > leader (which would only be notified via begin-quorum
> > > request).
> > > > > >> > However,
> > > > > >> > > > when persisting the quorum state, we would encode
> > leader-epoch
> > > > to
> > > > > >> N+1,
> > > > > >> > > > while the leaderId to be the older leader.
> > > > > >> > > >
> > > > > >> > > > But now thinking about this a bit more, I feel we should
> use
> > > two
> > > > > >> > separate
> > > > > >> > > > epochs, one for the "lates known" and one for the
> "current"
> > to
> > > > > pair
> > > > > >> > with
> > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch before
> the
> > > new
> > > > > >> leader
> > > > > >> > > is actually elected, right.
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> FetchQuorumRecords
> > > API
> > > > to
> > > > > >> > > indicate
> > > > > >> > > > > that the follower has fetched from an invalid offset and
> > > > should
> > > > > >> > > truncate
> > > > > >> > > > to
> > > > > >> > > > > the offset/epoch indicated in the response." Observers
> > can't
> > > > > >> truncate
> > > > > >> > > > their
> > > > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > I'm not sure if I understand your question? Observers
> should
> > > > still
> > > > > >> be
> > > > > >> > > able
> > > > > >> > > > to truncate their logs as well.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > Hmm, I thought only the quorum nodes have local logs and
> > > observers
> > > > > >> don't?
> > > > > >> > >
> > > > > >> > > > 107. "The leader will continue sending BeginQuorumEpoch to
> > > each
> > > > > >> known
> > > > > >> > > > voter
> > > > > >> > > > > until it has received its endorsement." If a voter is
> down
> > > > for a
> > > > > >> long
> > > > > >> > > > time,
> > > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> > overhead.
> > > > > >> > Similarly,
> > > > > >> > > > if a
> > > > > >> > > > > follower stops sending FetchQuorumRecords, does the
> leader
> > > > keep
> > > > > >> > sending
> > > > > >> > > > > BeginQuorumEpoch?
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > > > >> > begin-quorum-epoch
> > > > > >> > > > request is for voters to quickly get the new leader
> > > information;
> > > > > >> > however
> > > > > >> > > > even if they do not get them they can still eventually
> learn
> > > > about
> > > > > >> that
> > > > > >> > > > from others via gossiping FindQuorum. I think we can
> adjust
> > > the
> > > > > >> logic
> > > > > >> > to
> > > > > >> > > > e.g. exponential back-off or with a limited num.retries.
> > > > > >> > > >
> > > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > > >> FetchQuorumRecords
> > > > > >> > > > already, it means that follower already knows that the
> > broker
> > > is
> > > > > the
> > > > > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch;
> > > however
> > > > > it
> > > > > >> is
> > > > > >> > > > possible that after a follower sends FetchQuorumRecords
> > > already,
> > > > > >> > suddenly
> > > > > >> > > > it stops send it (possibly because it learned about a
> higher
> > > > epoch
> > > > > >> > > leader),
> > > > > >> > > > and hence this broker may be a "zombie" leader and we
> > propose
> > > to
> > > > > use
> > > > > >> > the
> > > > > >> > > > fetch.timeout to let the leader to try to verify if it has
> > > > already
> > > > > >> been
> > > > > >> > > > stale.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > It just seems that we should handle these two cases in a
> > > > consistent
> > > > > >> way?
> > > > > >> > >
> > > > > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords
> > from a
> > > > > >> follower
> > > > > >> > could mean that we no longer needs to send BeginQuorumEpoch
> > > anymore
> > > > > ---
> > > > > >> and
> > > > > >> > it is already part of our current implementations in
> > > > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > >> >
> > > > > >> >
> > > > > >> > > Thanks,
> > > > > >> > >
> > > > > >> > > Jun
> > > > > >> > >
> > > > > >> > > >
> > > > > >> > > > >
> > > > > >> > > > > Jun
> > > > > >> > > > >
> > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > >> >
> > > > > >> > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > Hello Leonard,
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks for your comments, I'm relying in line below:
> > > > > >> > > > > >
> > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > > > > >> > wge@confluent.io>
> > > > > >> > > > > > wrote:
> > > > > >> > > > > >
> > > > > >> > > > > > > Hi Kafka developers,
> > > > > >> > > > > > >
> > > > > >> > > > > > > It's great to see this proposal and it took me some
> > time
> > > > to
> > > > > >> > finish
> > > > > >> > > > > > reading
> > > > > >> > > > > > > it.
> > > > > >> > > > > > >
> > > > > >> > > > > > > And I have the following questions about the
> Proposal:
> > > > > >> > > > > > >
> > > > > >> > > > > > >    - How do we plan to test this design to ensure
> its
> > > > > >> > correctness?
> > > > > >> > > Or
> > > > > >> > > > > > more
> > > > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’
> based
> > > > model
> > > > > >> is
> > > > > >> > > > > > functional
> > > > > >> > > > > > > and
> > > > > >> > > > > > >    correct given that it is different from the
> > original
> > > > RAFT
> > > > > >> > > > > > implementation
> > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > We have two planned verifications on the correctness
> and
> > > > > >> liveness
> > > > > >> > of
> > > > > >> > > > the
> > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > > > > >> > > > > >
> > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > >> > > > > >
> > > > > >> > > > > >    - Have we considered any sensible defaults for the
> > > > > >> > configuration,
> > > > > >> > > > i.e.
> > > > > >> > > > > > >    all the election timeout, fetch time out, etc.?
> Or
> > we
> > > > > want
> > > > > >> to
> > > > > >> > > > leave
> > > > > >> > > > > > > this to
> > > > > >> > > > > > >    a later stage when we do the performance testing,
> > > etc.
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > This is a good question, the reason we did not set any
> > > > default
> > > > > >> > values
> > > > > >> > > > for
> > > > > >> > > > > > the timeout configurations is that we think it may
> take
> > > some
> > > > > >> > > > benchmarking
> > > > > >> > > > > > experiments to get these defaults right. Some
> high-level
> > > > > >> principles
> > > > > >> > > to
> > > > > >> > > > > > consider: 1) the fetch.timeout should be around the
> same
> > > > scale
> > > > > >> with
> > > > > >> > > zk
> > > > > >> > > > > > session timeout, which is now 18 seconds by default --
> > in
> > > > > >> practice
> > > > > >> > > > we've
> > > > > >> > > > > > seen unstable networks having more than 10 secs of
> > > transient
> > > > > >> > > > > connectivity,
> > > > > >> > > > > > 2) the election.timeout, however, should be smaller
> than
> > > the
> > > > > >> fetch
> > > > > >> > > > > timeout
> > > > > >> > > > > > as is also suggested as a practical optimization in
> > > > > literature:
> > > > > >> > > > > >
> > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > >> > > > > >
> > > > > >> > > > > > Some more discussions can be found here:
> > > > > >> > > > > >
> > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > >    - Have we considered piggybacking
> > `BeginQuorumEpoch`
> > > > with
> > > > > >> the
> > > > > >> > `
> > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing something
> > > > obvious
> > > > > >> but
> > > > > >> > I
> > > > > >> > > am
> > > > > >> > > > > > just
> > > > > >> > > > > > >    wondering why don’t we just use the `FindQuorum`
> > and
> > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader ->
> > > other
> > > > > >> voter
> > > > > >> > > > > > followers, while FindQuorum / Fetch are sent from
> > follower
> > > > to
> > > > > >> > leader.
> > > > > >> > > > > > Arguably one can eventually realize the new leader and
> > > epoch
> > > > > via
> > > > > >> > > > > gossiping
> > > > > >> > > > > > FindQuorum, but that could in practice require a long
> > > delay.
> > > > > >> > Having a
> > > > > >> > > > > > leader -> other voters request helps the new leader
> > epoch
> > > to
> > > > > be
> > > > > >> > > > > propagated
> > > > > >> > > > > > faster under a pull model.
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > >    - And about the `FetchQuorumRecords` response
> > schema,
> > > > in
> > > > > >> the
> > > > > >> > > > > `Records`
> > > > > >> > > > > > >    field of the response, is it just one record or
> all
> > > the
> > > > > >> > records
> > > > > >> > > > > > starting
> > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> efficient
> > > if
> > > > we
> > > > > >> sent
> > > > > >> > > all
> > > > > >> > > > > the
> > > > > >> > > > > > >    records during the bootstrapping of the brokers.
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> > > > starting
> > > > > >> > offset
> > > > > >> > > of
> > > > > >> > > > > the
> > > > > >> > > > > > batch of records.
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > >    - Regarding the disruptive broker issues, does
> our
> > > pull
> > > > > >> based
> > > > > >> > > > model
> > > > > >> > > > > > >    suffer from it? If so, have we considered the
> > > Pre-Vote
> > > > > >> stage?
> > > > > >> > If
> > > > > >> > > > > not,
> > > > > >> > > > > > > why?
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > The disruptive broker is stated in the original Raft
> > paper
> > > > > >> which is
> > > > > >> > > the
> > > > > >> > > > > > result of the push model design. Our analysis showed
> > that
> > > > with
> > > > > >> the
> > > > > >> > > pull
> > > > > >> > > > > > model it is no longer an issue.
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > > Thanks a lot for putting this up, and I hope that my
> > > > > questions
> > > > > >> > can
> > > > > >> > > be
> > > > > >> > > > > of
> > > > > >> > > > > > > some value to make this KIP better.
> > > > > >> > > > > > >
> > > > > >> > > > > > > Hope to hear from you soon!
> > > > > >> > > > > > >
> > > > > >> > > > > > > Best wishes,
> > > > > >> > > > > > > Leonard
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > > >> cmccabe@apache.org
> > > > > >> > >
> > > > > >> > > > > wrote:
> > > > > >> > > > > > >
> > > > > >> > > > > > > > Hi Jason,
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > I haven't had a chance to read in detail, but I
> read
> > > the
> > > > > >> > outline
> > > > > >> > > > and
> > > > > >> > > > > a
> > > > > >> > > > > > > few
> > > > > >> > > > > > > > things jumped out at me.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > First, for every epoch that is 32 bits rather than
> > > 64, I
> > > > > >> sort
> > > > > >> > of
> > > > > >> > > > > wonder
> > > > > >> > > > > > > if
> > > > > >> > > > > > > > that's a good long-term choice.  I keep reading
> > about
> > > > > stuff
> > > > > >> > like
> > > > > >> > > > > this:
> > > > > >> > > > > > > >
> > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > .
> > > > > >> > > Obviously,
> > > > > >> > > > > > that
> > > > > >> > > > > > > > JIRA is about zxid, which increments much faster
> > than
> > > we
> > > > > >> expect
> > > > > >> > > > these
> > > > > >> > > > > > > > leader epochs to, but it would still be good to
> see
> > > some
> > > > > >> rough
> > > > > >> > > > > > > calculations
> > > > > >> > > > > > > > about how long 32 bits (or really, 31 bits) will
> > last
> > > us
> > > > > in
> > > > > >> the
> > > > > >> > > > cases
> > > > > >> > > > > > > where
> > > > > >> > > > > > > > we're using it, and what the space savings we're
> > > getting
> > > > > >> really
> > > > > >> > > is.
> > > > > >> > > > > It
> > > > > >> > > > > > > > seems like in most cases the tradeoff may not be
> > worth
> > > > it?
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Another thing I've been thinking about is how we
> do
> > > > > >> > > > bootstrapping.  I
> > > > > >> > > > > > > > would prefer to be in a world where formatting a
> new
> > > > Kafka
> > > > > >> node
> > > > > >> > > > was a
> > > > > >> > > > > > > first
> > > > > >> > > > > > > > class operation explicitly initiated by the admin,
> > > > rather
> > > > > >> than
> > > > > >> > > > > > something
> > > > > >> > > > > > > > that happened implicitly when you started up the
> > > broker
> > > > > and
> > > > > >> > > things
> > > > > >> > > > > > > "looked
> > > > > >> > > > > > > > blank."
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > The first problem is that things can "look blank"
> > > > > >> accidentally
> > > > > >> > if
> > > > > >> > > > the
> > > > > >> > > > > > > > storage system is having a bad day.  Clearly in
> the
> > > > > non-Raft
> > > > > >> > > world,
> > > > > >> > > > > > this
> > > > > >> > > > > > > > leads to data loss if the broker that is
> (re)started
> > > > this
> > > > > >> way
> > > > > >> > was
> > > > > >> > > > the
> > > > > >> > > > > > > > leader for some partitions.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > The second problem is that we have a bit of a
> > chicken
> > > > and
> > > > > >> egg
> > > > > >> > > > problem
> > > > > >> > > > > > > with
> > > > > >> > > > > > > > certain configuration keys.  For example, maybe
> you
> > > want
> > > > > to
> > > > > >> > > > configure
> > > > > >> > > > > > > some
> > > > > >> > > > > > > > connection security settings in your cluster, but
> > you
> > > > > don't
> > > > > >> > want
> > > > > >> > > > them
> > > > > >> > > > > > to
> > > > > >> > > > > > > > ever be stored in a plaintext config file.  (For
> > > > example,
> > > > > >> SCRAM
> > > > > >> > > > > > > passwords,
> > > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > > configuration,
> > > > > >> but
> > > > > >> > > > that
> > > > > >> > > > > > > brings
> > > > > >> > > > > > > > up the chicken and egg problem.  The broker needs
> to
> > > be
> > > > > >> > > configured
> > > > > >> > > > to
> > > > > >> > > > > > > know
> > > > > >> > > > > > > > how to talk to you, but you need to configure it
> > > before
> > > > > you
> > > > > >> can
> > > > > >> > > > talk
> > > > > >> > > > > to
> > > > > >> > > > > > > > it.  Using an external secret manager like Vault
> is
> > > one
> > > > > way
> > > > > >> to
> > > > > >> > > > solve
> > > > > >> > > > > > > this,
> > > > > >> > > > > > > > but not everyone uses an external secret manager.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > quorum.voters seems like a similar configuration
> > key.
> > > > In
> > > > > >> the
> > > > > >> > > > current
> > > > > >> > > > > > > KIP,
> > > > > >> > > > > > > > this is only read if there is no other
> configuration
> > > > > >> specifying
> > > > > >> > > the
> > > > > >> > > > > > > quorum
> > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> > > wouldn't
> > > > > need
> > > > > >> > this
> > > > > >> > > > key
> > > > > >> > > > > > > > because we could assume that there was always
> quorum
> > > > > >> > information
> > > > > >> > > > > stored
> > > > > >> > > > > > > > locally.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > best,
> > > > > >> > > > > > > > Colin
> > > > > >> > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson
> > wrote:
> > > > > >> > > > > > > > > Hi All,
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > >> > > > > > > > .
> > > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > > ultimately
> > > > > >> replace
> > > > > >> > > > > > Zookeeper
> > > > > >> > > > > > > > > as
> > > > > >> > > > > > > > > documented in KIP-500. Please take a look and
> > share
> > > > your
> > > > > >> > > > thoughts.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > A few minor notes to set the stage a little bit:
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > - This KIP does not specify the structure of the
> > > > > messages
> > > > > >> > used
> > > > > >> > > to
> > > > > >> > > > > > > > represent
> > > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> > internal
> > > > API
> > > > > >> that
> > > > > >> > > will
> > > > > >> > > > > be
> > > > > >> > > > > > > used
> > > > > >> > > > > > > > > by the controller. Expect these to come in later
> > > > > >> proposals.
> > > > > >> > > Here
> > > > > >> > > > we
> > > > > >> > > > > > are
> > > > > >> > > > > > > > > primarily concerned with the replication
> protocol
> > > and
> > > > > >> basic
> > > > > >> > > > > > operational
> > > > > >> > > > > > > > > mechanics.
> > > > > >> > > > > > > > > - We expect many details to change as we get
> > closer
> > > to
> > > > > >> > > > integration
> > > > > >> > > > > > with
> > > > > >> > > > > > > > > the controller. Any changes we make will be made
> > > > either
> > > > > as
> > > > > >> > > > > amendments
> > > > > >> > > > > > > to
> > > > > >> > > > > > > > > this KIP or, in the case of larger changes, as
> new
> > > > > >> proposals.
> > > > > >> > > > > > > > > - We have a prototype implementation which I
> will
> > > put
> > > > > >> online
> > > > > >> > > > within
> > > > > >> > > > > > the
> > > > > >> > > > > > > > > next week which may help in understanding some
> > > > details.
> > > > > It
> > > > > >> > has
> > > > > >> > > > > > > diverged a
> > > > > >> > > > > > > > > little bit from our proposal, so I am taking a
> > > little
> > > > > >> time to
> > > > > >> > > > bring
> > > > > >> > > > > > it
> > > > > >> > > > > > > in
> > > > > >> > > > > > > > > line. I'll post an update to this thread when it
> > is
> > > > > >> available
> > > > > >> > > for
> > > > > >> > > > > > > review.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > Finally, I want to mention that this proposal
> was
> > > > > drafted
> > > > > >> by
> > > > > >> > > > > myself,
> > > > > >> > > > > > > > Boyang
> > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > Thanks,
> > > > > >> > > > > > > > > Jason
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > --
> > > > > >> > > > > > > Leonard Ge
> > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > --
> > > > > >> > > > > > -- Guozhang
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > --
> > > > > >> > > > -- Guozhang
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >> >
> > > > > >> > --
> > > > > >> > -- Guozhang
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Jun,

> Thanks for the response. Should we make clusterId a nullable field
consistently in all new requests?

Yes, makes sense. I updated the proposal.

-Jason

On Fri, Jul 31, 2020 at 4:17 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Jason,
>
> Thanks for the response. Should we make clusterId a nullable field
> consistently in all new requests?
>
> Jun
>
> On Wed, Jul 29, 2020 at 12:20 PM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Jun,
> >
> > I added a section on "Cluster Bootstrapping" which discusses clusterId
> > generation and the process through which brokers find the current leader.
> > The quick summary is that the first controller will be responsible for
> > generating the clusterId and persisting it in the metadata log. Before
> the
> > first leader has been elected, quorum APIs will skip clusterId
> validation.
> > This seems reasonable since this is primarily intended to prevent the
> > damage from misconfiguration after a cluster has been running for some
> > time. Upon startup, brokers begin by sending Fetch requests to find the
> > current leader. This will include the cluster.id from meta.properties if
> > it
> > is present. The broker will shutdown immediately if it receives
> > INVALID_CLUSTER_ID from the Fetch response.
> >
> > I also added some details about our testing strategy, which you asked
> about
> > previously.
> >
> > Thanks,
> > Jason
> >
> > On Mon, Jul 27, 2020 at 10:46 PM Boyang Chen <reluctanthero104@gmail.com
> >
> > wrote:
> >
> > > On Mon, Jul 27, 2020 at 4:58 AM Unmesh Joshi <un...@gmail.com>
> > > wrote:
> > >
> > > > Just checked etcd and zookeeper code, and both support leader to step
> > > down
> > > > as a follower to make sure there are no two leaders if the leader has
> > > been
> > > > disconnected from the majority of the followers
> > > > For etcd this is https://github.com/etcd-io/etcd/issues/3866
> > > > For Zookeeper its
> https://issues.apache.org/jira/browse/ZOOKEEPER-1699
> > > > I was just thinking if it would be difficult to implement in the Pull
> > > based
> > > > model, but I guess not. It is possibly the same way ISR list is
> managed
> > > > currently, if leader of the controller quorum loses majority of the
> > > > followers, it should step down and become follower, that way, telling
> > > > client in time that it was disconnected from the quorum, and not keep
> > on
> > > > sending state metadata to clients.
> > > >
> > > > Thanks,
> > > > Unmesh
> > > >
> > > >
> > > > On Mon, Jul 27, 2020 at 9:31 AM Unmesh Joshi <un...@gmail.com>
> > > > wrote:
> > > >
> > > > > >>Could you clarify on this question? Which part of the raft group
> > > > doesn't
> > > > > >>know about leader dis-connection?
> > > > > The leader of the controller quorum is partitioned from the
> > controller
> > > > > cluster, and a different leader is elected for the remaining
> > controller
> > > > > cluster.
> > > >
> > > I see your concern. For KIP-595 implementation, since there is no
> regular
> > > heartbeats sent
> > > from the leader to the followers, we decided to piggy-back on the fetch
> > > timeout so that if the leader did not receive Fetch
> > > requests from a majority of the quorum for that amount of time, it
> would
> > > begin a new election and
> > > start sending VoteRequest to voter nodes in the cluster to understand
> the
> > > latest quorum. You could
> > > find more details in this section
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP595:ARaftProtocolfortheMetadataQuorum-Vote
> > > >
> > > .
> > >
> > >
> > > > > I think there are two things here,
> > > > > 1.  The old leader will not know if it's disconnected from the rest
> > of
> > > > the
> > > > > controller quorum cluster unless it receives BeginQuorumEpoch from
> > the
> > > > new
> > > > > leader. So it will keep on serving stale metadata to the clients
> > > > (Brokers,
> > > > > Producers and Consumers)
> > > > > 2. I assume, the Broker Leases will be managed on the controller
> > quorum
> > > > > leader. This partitioned leader will keep on tracking broker leases
> > it
> > > > has,
> > > > > while the new leader of the quorum will also start managing broker
> > > > leases.
> > > > > So while the quorum leader is partitioned, there will be two
> > membership
> > > > > views of the kafka brokers managed on two leaders.
> > > > > Unless broker heartbeats are also replicated as part of the Raft
> log,
> > > > > there is no way to solve this?
> > > > > I know LogCabin implementation does replicate client heartbeats. I
> > > > suspect
> > > > > that the same issue is there in Zookeeper, which does not replicate
> > > > client
> > > > > Ping requests..
> > > > >
> > > > > Thanks,
> > > > > Unmesh
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Jul 27, 2020 at 6:23 AM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > >> Thanks for the questions Unmesh!
> > > > >>
> > > > >> On Sun, Jul 26, 2020 at 6:18 AM Unmesh Joshi <
> unmeshjoshi@gmail.com
> > >
> > > > >> wrote:
> > > > >>
> > > > >> > Hi,
> > > > >> >
> > > > >> > In the FetchRequest Handling, how to make sure we handle
> scenarios
> > > > where
> > > > >> > the leader might have been disconnected from the cluster, but
> > > doesn't
> > > > >> know
> > > > >> > yet?
> > > > >> >
> > > > >> Could you clarify on this question? Which part of the raft group
> > > doesn't
> > > > >> know about leader
> > > > >> dis-connection?
> > > > >>
> > > > >>
> > > > >> > As discussed in the Raft Thesis section 6.4, the linearizable
> > > > semantics
> > > > >> of
> > > > >> > read requests is implemented in LogCabin by sending heartbeat to
> > > > >> followers
> > > > >> > and waiting till the heartbeats are successful to make sure that
> > the
> > > > >> leader
> > > > >> > is still the leader.
> > > > >> > I think for the controller quorum to make sure none of the
> > consumers
> > > > get
> > > > >> > stale data, it's important to have linearizable semantics? In
> the
> > > pull
> > > > >> > based model, the leader will need to wait for heartbeats from
> the
> > > > >> followers
> > > > >> > before returning each fetch request from the consumer then? Or
> do
> > we
> > > > >> need
> > > > >> > to introduce some other request?
> > > > >> > (Zookeeper does not have linearizable semantics for read
> requests,
> > > but
> > > > >> as
> > > > >> > of now all the kafka interactions are through writes and
> watches).
> > > > >> >
> > > > >> > This is a very good question. For our v1 implementation we are
> not
> > > > >> aiming
> > > > >> to guarantee linearizable read, which
> > > > >> would be considered as a follow-up effort. Note that today in
> Kafka
> > > > there
> > > > >> is no guarantee on the metadata freshness either,
> > > > >> so no regression is introduced.
> > > > >>
> > > > >>
> > > > >> > Thanks,
> > > > >> > Unmesh
> > > > >> >
> > > > >> > On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > >> >
> > > > >> > > Hi, Jason,
> > > > >> > >
> > > > >> > > Thanks for the reply.
> > > > >> > >
> > > > >> > > 101. Sounds good. Regarding clusterId, I am not sure storing
> it
> > in
> > > > the
> > > > >> > > metadata log is enough. For example, the vote request includes
> > > > >> clusterId.
> > > > >> > > So, no one can vote until they know the clusterId. Also, it
> > would
> > > be
> > > > >> > useful
> > > > >> > > to support the case when a voter completely loses its disk and
> > > needs
> > > > >> to
> > > > >> > > recover.
> > > > >> > >
> > > > >> > > 210. There is no longer a FindQuorum request. When a follower
> > > > >> restarts,
> > > > >> > how
> > > > >> > > does it discover the leader? Is that based on DescribeQuorum?
> It
> > > > >> would be
> > > > >> > > useful to document this.
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > > On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <
> > > jason@confluent.io
> > > > >
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Hi Jun,
> > > > >> > > >
> > > > >> > > > Thanks for the questions.
> > > > >> > > >
> > > > >> > > > 101. I am treating some of the bootstrapping problems as out
> > of
> > > > the
> > > > >> > scope
> > > > >> > > > of this KIP. I am working on a separate proposal which
> > addresses
> > > > >> > > > bootstrapping security credentials specifically. Here is a
> > rough
> > > > >> sketch
> > > > >> > > of
> > > > >> > > > how I am seeing it:
> > > > >> > > >
> > > > >> > > > 1. Dynamic broker configurations including encrypted
> passwords
> > > > will
> > > > >> be
> > > > >> > > > persisted in the metadata log and cached in the broker's
> > > > >> > > `meta.properties`
> > > > >> > > > file.
> > > > >> > > > 2. We will provide a tool which allows users to directly
> > > override
> > > > >> the
> > > > >> > > > values in `meta.properties` without requiring access to the
> > > > quorum.
> > > > >> > This
> > > > >> > > > can be used to bootstrap the credentials of the voter set
> > itself
> > > > >> before
> > > > >> > > the
> > > > >> > > > cluster has been started.
> > > > >> > > > 3. Some dynamic config changes will only be allowed when a
> > > broker
> > > > is
> > > > >> > > > online. For example, changing a truststore password
> > dynamically
> > > > >> would
> > > > >> > > > prevent that broker from being able to start if it were
> > offline
> > > > when
> > > > >> > the
> > > > >> > > > change was made.
> > > > >> > > > 4. I am still thinking a little bit about SCRAM credentials,
> > but
> > > > >> most
> > > > >> > > > likely they will be handled with an approach similar to
> > > > >> > > `meta.properties`.
> > > > >> > > >
> > > > >> > > > 101.3 As for the question about `clusterId`, I think the way
> > we
> > > > >> would
> > > > >> > do
> > > > >> > > > this is to have the first elected leader generate a UUID and
> > > write
> > > > >> it
> > > > >> > to
> > > > >> > > > the metadata log. Let me add some detail to the proposal
> about
> > > > this.
> > > > >> > > >
> > > > >> > > > A few additional answers below:
> > > > >> > > >
> > > > >> > > > 203. Yes, that is correct.
> > > > >> > > >
> > > > >> > > > 204. That is a good question. What happens in this case is
> > that
> > > > all
> > > > >> > > voters
> > > > >> > > > advance their epoch to the one designated by the candidate
> > even
> > > if
> > > > >> they
> > > > >> > > > reject its vote request. Assuming the candidate fails to be
> > > > elected,
> > > > >> > the
> > > > >> > > > election will be retried until a leader emerges.
> > > > >> > > >
> > > > >> > > > 205. I had some discussion with Colin offline about this
> > > problem.
> > > > I
> > > > >> > think
> > > > >> > > > the answer should be "yes," but it probably needs a little
> > more
> > > > >> > thought.
> > > > >> > > > Handling JBOD failures is tricky. For an observer, we can
> > > > replicate
> > > > >> the
> > > > >> > > > metadata log from scratch safely in a new log dir. But if
> the
> > > log
> > > > >> dir
> > > > >> > of
> > > > >> > > a
> > > > >> > > > voter fails, I do not think it is generally safe to start
> from
> > > an
> > > > >> empty
> > > > >> > > > state.
> > > > >> > > >
> > > > >> > > > 206. Yes, that is discussed in KIP-631 I believe.
> > > > >> > > >
> > > > >> > > > 207. Good suggestion. I will work on this.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Jason
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > >> > > >
> > > > >> > > > > Hi, Jason,
> > > > >> > > > >
> > > > >> > > > > Thanks for the updated KIP. Looks good overall. A few more
> > > > >> comments
> > > > >> > > > below.
> > > > >> > > > >
> > > > >> > > > > 101. I still don't see a section on bootstrapping related
> > > > issues.
> > > > >> It
> > > > >> > > > would
> > > > >> > > > > be useful to document if/how the following is supported.
> > > > >> > > > > 101.1 Currently, we support auto broker id generation. Is
> > this
> > > > >> > > supported
> > > > >> > > > > for bootstrap brokers?
> > > > >> > > > > 101.2 As Colin mentioned, sometimes we may need to load
> the
> > > > >> security
> > > > >> > > > > credentials to be broker before it can be connected to.
> > Could
> > > > you
> > > > >> > > > provide a
> > > > >> > > > > bit more detail on how this will work?
> > > > >> > > > > 101.3 Currently, we use ZK to generate clusterId on a new
> > > > cluster.
> > > > >> > With
> > > > >> > > > > Raft, how does every broker generate the same clusterId
> in a
> > > > >> > > distributed
> > > > >> > > > > way?
> > > > >> > > > >
> > > > >> > > > > 200. It would be useful to document if the various special
> > > > offsets
> > > > >> > (log
> > > > >> > > > > start offset, recovery point, HWM, etc) for the Raft log
> are
> > > > >> stored
> > > > >> > in
> > > > >> > > > the
> > > > >> > > > > same existing checkpoint files or not.
> > > > >> > > > > 200.1 Since the Raft log flushes every append, does that
> > allow
> > > > us
> > > > >> to
> > > > >> > > > > recover from a recovery point within the active segment or
> > do
> > > we
> > > > >> > still
> > > > >> > > > need
> > > > >> > > > > to scan the full segment including the recovery point? The
> > > > former
> > > > >> can
> > > > >> > > be
> > > > >> > > > > tricky since multiple records can fall into the same disk
> > page
> > > > >> and a
> > > > >> > > > > subsequent flush may corrupt a page with previously
> flushed
> > > > >> records.
> > > > >> > > > >
> > > > >> > > > > 201. Configurations.
> > > > >> > > > > 201.1 How do the Raft brokers get security related configs
> > for
> > > > >> inter
> > > > >> > > > broker
> > > > >> > > > > communication? Is that based on the existing
> > > > >> > > > > inter.broker.security.protocol?
> > > > >> > > > > 201.2 We have quorum.retry.backoff.max.ms and
> > > > >> > quorum.retry.backoff.ms,
> > > > >> > > > but
> > > > >> > > > > only quorum.election.backoff.max.ms. This seems a bit
> > > > >> inconsistent.
> > > > >> > > > >
> > > > >> > > > > 202. Metrics:
> > > > >> > > > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
> > > > >> > > OutboundQueueTimeMs:
> > > > >> > > > > Are those the same as existing totalTime,
> requestQueueTime,
> > > > >> > localTime,
> > > > >> > > > > responseQueueTime? Could we reuse the existing ones with
> the
> > > tag
> > > > >> > > > > request=[request-type]?
> > > > >> > > > > 202.2. Could you explain what InboundChannelSize and
> > > > >> > > OutboundChannelSize
> > > > >> > > > > are?
> > > > >> > > > > 202.3 ElectionLatencyMax/Avg: It seems that both should be
> > > > >> windowed?
> > > > >> > > > >
> > > > >> > > > > 203. Quorum State: I assume that LeaderId will be kept
> > > > >> consistently
> > > > >> > > with
> > > > >> > > > > LeaderEpoch. For example, if a follower transitions to
> > > candidate
> > > > >> and
> > > > >> > > > bumps
> > > > >> > > > > up LeaderEpoch, it will set leaderId to -1 and persist
> both
> > in
> > > > the
> > > > >> > > Quorum
> > > > >> > > > > state file. Is that correct?
> > > > >> > > > >
> > > > >> > > > > 204. I was thinking about a corner case when a Raft broker
> > is
> > > > >> > > partitioned
> > > > >> > > > > off. This broker will then be in a continuous loop of
> > bumping
> > > up
> > > > >> the
> > > > >> > > > leader
> > > > >> > > > > epoch, but failing to get enough votes. When the
> > partitioning
> > > is
> > > > >> > > removed,
> > > > >> > > > > this broker's high leader epoch will force a leader
> > election.
> > > I
> > > > >> > assume
> > > > >> > > > > other Raft brokers can immediately advance their leader
> > epoch
> > > > >> passing
> > > > >> > > the
> > > > >> > > > > already bumped epoch such that leader election won't be
> > > delayed.
> > > > >> Is
> > > > >> > > that
> > > > >> > > > > right?
> > > > >> > > > >
> > > > >> > > > > 205. In a JBOD setting, could we use the existing tool to
> > move
> > > > the
> > > > >> > Raft
> > > > >> > > > log
> > > > >> > > > > from one disk to another?
> > > > >> > > > >
> > > > >> > > > > 206. The KIP doesn't mention the local metadata store
> > derived
> > > > from
> > > > >> > the
> > > > >> > > > Raft
> > > > >> > > > > log. Will that be covered in a separate KIP?
> > > > >> > > > >
> > > > >> > > > > 207. Since this is a critical component. Could we add a
> > > section
> > > > on
> > > > >> > the
> > > > >> > > > > testing plan for correctness?
> > > > >> > > > >
> > > > >> > > > > 208. Performance. Do we plan to do group commit (e.g.
> buffer
> > > > >> pending
> > > > >> > > > > appends during a flush and then flush all accumulated
> > pending
> > > > >> records
> > > > >> > > > > together in the next flush) for better throughput?
> > > > >> > > > >
> > > > >> > > > > 209. "the leader can actually defer fsync until it knows
> > > > >> > "quorum.size -
> > > > >> > > > 1"
> > > > >> > > > > has get to a certain entry offset." Why is that
> > "quorum.size -
> > > > 1"
> > > > >> > > instead
> > > > >> > > > > of the majority of the quorum?
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <
> > > > >> jason@confluent.io>
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hi All,
> > > > >> > > > > >
> > > > >> > > > > > Just a quick update on the proposal. We have decided to
> > move
> > > > >> quorum
> > > > >> > > > > > reassignment to a separate KIP:
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > > > >> > > > > > .
> > > > >> > > > > > The way this ties into cluster bootstrapping is
> > complicated,
> > > > so
> > > > >> we
> > > > >> > > felt
> > > > >> > > > > we
> > > > >> > > > > > needed a bit more time for validation. That leaves the
> > core
> > > of
> > > > >> this
> > > > >> > > > > > proposal as quorum-based replication. If there are no
> > > further
> > > > >> > > comments,
> > > > >> > > > > we
> > > > >> > > > > > will plan to start a vote later this week.
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Jason
> > > > >> > > > > >
> > > > >> > > > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <
> > > > >> wangguoz@gmail.com
> > > > >> > >
> > > > >> > > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > > > @Jun Rao <ju...@gmail.com>
> > > > >> > > > > > >
> > > > >> > > > > > > Regarding your comment about log compaction. After
> some
> > > > >> > deep-diving
> > > > >> > > > > into
> > > > >> > > > > > > this we've decided to propose a new snapshot-based log
> > > > >> cleaning
> > > > >> > > > > mechanism
> > > > >> > > > > > > which would be used to replace the current compaction
> > > > >> mechanism
> > > > >> > for
> > > > >> > > > > this
> > > > >> > > > > > > meta log. A new KIP will be proposed specifically for
> > this
> > > > >> idea.
> > > > >> > > > > > >
> > > > >> > > > > > > All,
> > > > >> > > > > > >
> > > > >> > > > > > > I've updated the KIP wiki a bit updating one config "
> > > > >> > > > > > > election.jitter.max.ms"
> > > > >> > > > > > > to "election.backoff.max.ms" to make it more clear
> > about
> > > > the
> > > > >> > > usage:
> > > > >> > > > > the
> > > > >> > > > > > > configured value will be the upper bound of the binary
> > > > >> > exponential
> > > > >> > > > > > backoff
> > > > >> > > > > > > time after a failed election, before starting a new
> one.
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > Guozhang
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> > > > >> > > > > reluctanthero104@gmail.com>
> > > > >> > > > > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Thanks for the suggestions Guozhang.
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
> > > > >> > > wangguoz@gmail.com>
> > > > >> > > > > > > wrote:
> > > > >> > > > > > > >
> > > > >> > > > > > > > > Hello Boyang,
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Thanks for the updated information. A few
> questions
> > > > here:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > 1) Should the quorum-file also update to support
> > > > >> multi-raft?
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > I'm neutral about this, as we don't know yet how
> the
> > > > >> > multi-raft
> > > > >> > > > > > modules
> > > > >> > > > > > > > would behave. If
> > > > >> > > > > > > > we have different threads operating different raft
> > > groups,
> > > > >> > > > > > consolidating
> > > > >> > > > > > > > the `checkpoint` files seems
> > > > >> > > > > > > > not reasonable. We could always add
> > `multi-quorum-file`
> > > > >> later
> > > > >> > if
> > > > >> > > > > > > possible.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 2) In the previous proposal, there's fields in the
> > > > >> > > > FetchQuorumRecords
> > > > >> > > > > > > like
> > > > >> > > > > > > > > latestDirtyOffset, is that dropped intentionally?
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > I dropped the latestDirtyOffset since it is
> > associated
> > > > >> with
> > > > >> > the
> > > > >> > > > log
> > > > >> > > > > > > > compaction discussion. This is beyond this KIP scope
> > and
> > > > we
> > > > >> > could
> > > > >> > > > > > > > potentially get a separate KIP to talk about it.
> > > > >> > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > > > > 3) I think we also need to elaborate a bit more
> > > details
> > > > >> > > regarding
> > > > >> > > > > > when
> > > > >> > > > > > > to
> > > > >> > > > > > > > > send metadata request and discover-brokers;
> > currently
> > > we
> > > > >> only
> > > > >> > > > > > discussed
> > > > >> > > > > > > > > during bootstrap how these requests would be
> sent. I
> > > > think
> > > > >> > the
> > > > >> > > > > > > following
> > > > >> > > > > > > > > scenarios would also need these requests
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > 3.a) As long as a broker does not know the current
> > > > quorum
> > > > >> > > > > (including
> > > > >> > > > > > > the
> > > > >> > > > > > > > > leader and the voters), it should continue
> > > periodically
> > > > >> ask
> > > > >> > > other
> > > > >> > > > > > > brokers
> > > > >> > > > > > > > > via "metadata.
> > > > >> > > > > > > > > 3.b) As long as a broker does not know all the
> > current
> > > > >> quorum
> > > > >> > > > > voter's
> > > > >> > > > > > > > > connections, it should continue periodically ask
> > other
> > > > >> > brokers
> > > > >> > > > via
> > > > >> > > > > > > > > "discover-brokers".
> > > > >> > > > > > > > > 3.c) When the leader's fetch timeout elapsed, it
> > > should
> > > > >> send
> > > > >> > > > > metadata
> > > > >> > > > > > > > > request.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Make sense, will add to the KIP.
> > > > >> > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Guozhang
> > > > >> > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > > >> > > > > > > reluctanthero104@gmail.com>
> > > > >> > > > > > > > > wrote:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > > Hey all,
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > follow-up on the previous email, we made some
> more
> > > > >> updates:
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > 1. The Alter/DescribeQuorum RPCs are also
> > > > re-structured
> > > > >> to
> > > > >> > > use
> > > > >> > > > > > > > > multi-raft.
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > 2. We add observer status into the
> > > > >> DescribeQuorumResponse
> > > > >> > as
> > > > >> > > we
> > > > >> > > > > see
> > > > >> > > > > > > it
> > > > >> > > > > > > > > is a
> > > > >> > > > > > > > > > low hanging fruit which is very useful for user
> > > > >> debugging
> > > > >> > and
> > > > >> > > > > > > > > reassignment.
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > 3. The FindQuorum RPC is replaced with
> > > DiscoverBrokers
> > > > >> RPC,
> > > > >> > > > which
> > > > >> > > > > > is
> > > > >> > > > > > > > > purely
> > > > >> > > > > > > > > > in charge of discovering broker connections in a
> > > > gossip
> > > > >> > > manner.
> > > > >> > > > > The
> > > > >> > > > > > > > > quorum
> > > > >> > > > > > > > > > leader discovery is piggy-back on the Metadata
> RPC
> > > for
> > > > >> the
> > > > >> > > > topic
> > > > >> > > > > > > > > partition
> > > > >> > > > > > > > > > leader, which in our case is the single metadata
> > > > >> partition
> > > > >> > > for
> > > > >> > > > > the
> > > > >> > > > > > > > > version
> > > > >> > > > > > > > > > one.
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > Let me know if you have any questions.
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > Boyang
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > >> > > > > > > > reluctanthero104@gmail.com>
> > > > >> > > > > > > > > > wrote:
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > > Hey all,
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > Thanks for the great discussions so far. I'm
> > > posting
> > > > >> some
> > > > >> > > KIP
> > > > >> > > > > > > updates
> > > > >> > > > > > > > > > from
> > > > >> > > > > > > > > > > our working group discussion:
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > 1. We will be changing the core RPCs from
> > > > single-raft
> > > > >> API
> > > > >> > > to
> > > > >> > > > > > > > > multi-raft.
> > > > >> > > > > > > > > > > This means all protocols will be "batch" in
> the
> > > > first
> > > > >> > > > version,
> > > > >> > > > > > but
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > KIP
> > > > >> > > > > > > > > > > itself only illustrates the design for a
> single
> > > > >> metadata
> > > > >> > > > topic
> > > > >> > > > > > > > > partition.
> > > > >> > > > > > > > > > > The reason is to "keep the door open" for
> future
> > > > >> > extensions
> > > > >> > > > of
> > > > >> > > > > > this
> > > > >> > > > > > > > > piece
> > > > >> > > > > > > > > > > of module such as a sharded controller or
> > general
> > > > >> quorum
> > > > >> > > > based
> > > > >> > > > > > > topic
> > > > >> > > > > > > > > > > replication, beyond the current Kafka
> > replication
> > > > >> > protocol.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > 2. We will piggy-back on the current Kafka
> Fetch
> > > API
> > > > >> > > instead
> > > > >> > > > of
> > > > >> > > > > > > > > inventing
> > > > >> > > > > > > > > > > a new FetchQuorumRecords RPC. The motivation
> is
> > > > about
> > > > >> the
> > > > >> > > > same
> > > > >> > > > > as
> > > > >> > > > > > > #1
> > > > >> > > > > > > > as
> > > > >> > > > > > > > > > > well as making the integration work easier,
> > > instead
> > > > of
> > > > >> > > > letting
> > > > >> > > > > > two
> > > > >> > > > > > > > > > similar
> > > > >> > > > > > > > > > > RPCs diverge.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of
> > only
> > > > >> > sending
> > > > >> > > > the
> > > > >> > > > > > > > request
> > > > >> > > > > > > > > to
> > > > >> > > > > > > > > > > the most caught-up voter, we shall broadcast
> the
> > > > >> > > information
> > > > >> > > > to
> > > > >> > > > > > all
> > > > >> > > > > > > > > > voters,
> > > > >> > > > > > > > > > > with a sorted voter list in descending order
> of
> > > > their
> > > > >> > > > > > corresponding
> > > > >> > > > > > > > > > > replicated offset. In this way, the top voter
> > will
> > > > >> > become a
> > > > >> > > > > > > candidate
> > > > >> > > > > > > > > > > immediately, while the other voters shall wait
> > for
> > > > an
> > > > >> > > > > exponential
> > > > >> > > > > > > > > > back-off
> > > > >> > > > > > > > > > > to trigger elections, which helps ensure the
> top
> > > > voter
> > > > >> > gets
> > > > >> > > > > > > elected,
> > > > >> > > > > > > > > and
> > > > >> > > > > > > > > > > the election eventually happens when the top
> > voter
> > > > is
> > > > >> not
> > > > >> > > > > > > responsive.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > Please see the updated KIP and post any
> > questions
> > > or
> > > > >> > > concerns
> > > > >> > > > > on
> > > > >> > > > > > > the
> > > > >> > > > > > > > > > > mailing thread.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > Boyang
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <
> > > > >> jun@confluent.io
> > > > >> > >
> > > > >> > > > > wrote:
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > >> Hi, Guozhang and Jason,
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > > >> Thanks for the reply. A couple of more
> replies.
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > > >> 102. Still not sure about this. How is the
> > > > tombstone
> > > > >> > issue
> > > > >> > > > > > > addressed
> > > > >> > > > > > > > > in
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> non-voter and the observer.  They can die at
> > any
> > > > >> point
> > > > >> > and
> > > > >> > > > > > restart
> > > > >> > > > > > > > at
> > > > >> > > > > > > > > an
> > > > >> > > > > > > > > > >> arbitrary later time, and the advancing of
> the
> > > > >> > firstDirty
> > > > >> > > > > offset
> > > > >> > > > > > > and
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > >> removal of the tombstone can happen
> > > independently.
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > > >> 106. I agree that it would be less confusing
> if
> > > we
> > > > >> used
> > > > >> > > > > "epoch"
> > > > >> > > > > > > > > instead
> > > > >> > > > > > > > > > of
> > > > >> > > > > > > > > > >> "leader epoch" consistently.
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > > >> Jun
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang
> <
> > > > >> > > > > > wangguoz@gmail.com>
> > > > >> > > > > > > > > > wrote:
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
> > > > >> > > jun@confluent.io
> > > > >> > > > >
> > > > >> > > > > > > wrote:
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > > Hi, Guozhang,
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > Thanks for the reply. A few more replies
> > > > inlined
> > > > >> > > below.
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang
> > Wang
> > > <
> > > > >> > > > > > > > wangguoz@gmail.com>
> > > > >> > > > > > > > > > >> wrote:
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > > Hello Jun,
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > Thanks for your comments! I'm replying
> > > inline
> > > > >> > below:
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun
> Rao <
> > > > >> > > > > jun@confluent.io
> > > > >> > > > > > >
> > > > >> > > > > > > > > wrote:
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > >> > > > > > > > > > >> > > > > 101.1 Currently, we support auto
> broker
> > > id
> > > > >> > > > generation.
> > > > >> > > > > > Is
> > > > >> > > > > > > > this
> > > > >> > > > > > > > > > >> > > supported
> > > > >> > > > > > > > > > >> > > > > for bootstrap brokers?
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > The vote ids would just be the broker
> > ids.
> > > > >> > > > > > > "bootstrap.servers"
> > > > >> > > > > > > > > > >> would be
> > > > >> > > > > > > > > > >> > > > similar to what client configs have
> > today,
> > > > >> where
> > > > >> > > > > > > > "quorum.voters"
> > > > >> > > > > > > > > > >> would
> > > > >> > > > > > > > > > >> > be
> > > > >> > > > > > > > > > >> > > > pre-defined config values.
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > My question was on the auto generated
> > broker
> > > > id.
> > > > >> > > > > Currently,
> > > > >> > > > > > > the
> > > > >> > > > > > > > > > broker
> > > > >> > > > > > > > > > >> > can
> > > > >> > > > > > > > > > >> > > choose to have its broker Id auto
> > generated.
> > > > The
> > > > >> > > > > generation
> > > > >> > > > > > is
> > > > >> > > > > > > > > done
> > > > >> > > > > > > > > > >> > through
> > > > >> > > > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK,
> > it's
> > > > not
> > > > >> > clear
> > > > >> > > > how
> > > > >> > > > > > the
> > > > >> > > > > > > > > > broker
> > > > >> > > > > > > > > > >> id
> > > > >> > > > > > > > > > >> > is
> > > > >> > > > > > > > > > >> > > auto generated. "quorum.voters" also
> can't
> > be
> > > > set
> > > > >> > > > > statically
> > > > >> > > > > > > if
> > > > >> > > > > > > > > > broker
> > > > >> > > > > > > > > > >> > ids
> > > > >> > > > > > > > > > >> > > are auto generated.
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > Jason has explained some ideas that we've
> > > > >> discussed
> > > > >> > so
> > > > >> > > > > far,
> > > > >> > > > > > > the
> > > > >> > > > > > > > > > >> reason we
> > > > >> > > > > > > > > > >> > intentional did not include them so far is
> > that
> > > > we
> > > > >> > feel
> > > > >> > > it
> > > > >> > > > > is
> > > > >> > > > > > > > > out-side
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > scope of KIP-595. Under the umbrella of
> > KIP-500
> > > > we
> > > > >> > > should
> > > > >> > > > > > > > definitely
> > > > >> > > > > > > > > > >> > address them though.
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > On the high-level, our belief is that
> > "joining
> > > a
> > > > >> > quorum"
> > > > >> > > > and
> > > > >> > > > > > > > > "joining
> > > > >> > > > > > > > > > >> (or
> > > > >> > > > > > > > > > >> > more specifically, registering brokers in)
> > the
> > > > >> > cluster"
> > > > >> > > > > would
> > > > >> > > > > > be
> > > > >> > > > > > > > > > >> > de-coupled a bit, where the former should
> be
> > > > >> completed
> > > > >> > > > > before
> > > > >> > > > > > we
> > > > >> > > > > > > > do
> > > > >> > > > > > > > > > the
> > > > >> > > > > > > > > > >> > latter. More specifically, assuming the
> > quorum
> > > is
> > > > >> > > already
> > > > >> > > > up
> > > > >> > > > > > and
> > > > >> > > > > > > > > > >> running,
> > > > >> > > > > > > > > > >> > after the newly started broker found the
> > leader
> > > > of
> > > > >> the
> > > > >> > > > > quorum
> > > > >> > > > > > it
> > > > >> > > > > > > > can
> > > > >> > > > > > > > > > >> send a
> > > > >> > > > > > > > > > >> > specific RegisterBroker request including
> its
> > > > >> > listener /
> > > > >> > > > > > > protocol
> > > > >> > > > > > > > /
> > > > >> > > > > > > > > > etc,
> > > > >> > > > > > > > > > >> > and upon handling it the leader can send
> back
> > > the
> > > > >> > > uniquely
> > > > >> > > > > > > > generated
> > > > >> > > > > > > > > > >> broker
> > > > >> > > > > > > > > > >> > id to the new broker, while also executing
> > the
> > > > >> > > > > > "startNewBroker"
> > > > >> > > > > > > > > > >> callback as
> > > > >> > > > > > > > > > >> > the controller.
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > > > 102. Log compaction. One weak spot of
> > log
> > > > >> > > compaction
> > > > >> > > > > is
> > > > >> > > > > > > for
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > >> > > consumer
> > > > >> > > > > > > > > > >> > > > to
> > > > >> > > > > > > > > > >> > > > > deal with deletes. When a key is
> > deleted,
> > > > >> it's
> > > > >> > > > > retained
> > > > >> > > > > > > as a
> > > > >> > > > > > > > > > >> > tombstone
> > > > >> > > > > > > > > > >> > > > > first and then physically removed.
> If a
> > > > >> client
> > > > >> > > > misses
> > > > >> > > > > > the
> > > > >> > > > > > > > > > >> tombstone
> > > > >> > > > > > > > > > >> > > > > (because it's physically removed), it
> > may
> > > > >> not be
> > > > >> > > > able
> > > > >> > > > > to
> > > > >> > > > > > > > > update
> > > > >> > > > > > > > > > >> its
> > > > >> > > > > > > > > > >> > > > > metadata properly. The way we solve
> > this
> > > in
> > > > >> > Kafka
> > > > >> > > is
> > > > >> > > > > > based
> > > > >> > > > > > > > on
> > > > >> > > > > > > > > a
> > > > >> > > > > > > > > > >> > > > > configuration (
> > > > >> log.cleaner.delete.retention.ms)
> > > > >> > > and
> > > > >> > > > > we
> > > > >> > > > > > > > > expect a
> > > > >> > > > > > > > > > >> > > consumer
> > > > >> > > > > > > > > > >> > > > > having seen an old key to finish
> > reading
> > > > the
> > > > >> > > > deletion
> > > > >> > > > > > > > > tombstone
> > > > >> > > > > > > > > > >> > within
> > > > >> > > > > > > > > > >> > > > that
> > > > >> > > > > > > > > > >> > > > > time. There is no strong guarantee
> for
> > > that
> > > > >> > since
> > > > >> > > a
> > > > >> > > > > > broker
> > > > >> > > > > > > > > could
> > > > >> > > > > > > > > > >> be
> > > > >> > > > > > > > > > >> > > down
> > > > >> > > > > > > > > > >> > > > > for a long time. It would be better
> if
> > we
> > > > can
> > > > >> > > have a
> > > > >> > > > > > more
> > > > >> > > > > > > > > > reliable
> > > > >> > > > > > > > > > >> > way
> > > > >> > > > > > > > > > >> > > of
> > > > >> > > > > > > > > > >> > > > > dealing with deletes.
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > We propose to capture this in the
> > > > >> > "FirstDirtyOffset"
> > > > >> > > > > field
> > > > >> > > > > > > of
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > >> > quorum
> > > > >> > > > > > > > > > >> > > > record fetch response: the offset is
> the
> > > > >> maximum
> > > > >> > > > offset
> > > > >> > > > > > that
> > > > >> > > > > > > > log
> > > > >> > > > > > > > > > >> > > compaction
> > > > >> > > > > > > > > > >> > > > has reached up to. If the follower has
> > > > fetched
> > > > >> > > beyond
> > > > >> > > > > this
> > > > >> > > > > > > > > offset
> > > > >> > > > > > > > > > it
> > > > >> > > > > > > > > > >> > > means
> > > > >> > > > > > > > > > >> > > > itself is safe hence it has seen all
> > > records
> > > > >> up to
> > > > >> > > > that
> > > > >> > > > > > > > offset.
> > > > >> > > > > > > > > On
> > > > >> > > > > > > > > > >> > > getting
> > > > >> > > > > > > > > > >> > > > the response, the follower can then
> > decide
> > > if
> > > > >> its
> > > > >> > > end
> > > > >> > > > > > offset
> > > > >> > > > > > > > > > >> actually
> > > > >> > > > > > > > > > >> > > below
> > > > >> > > > > > > > > > >> > > > that dirty offset (and hence may miss
> > some
> > > > >> > > > tombstones).
> > > > >> > > > > If
> > > > >> > > > > > > > > that's
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > > case:
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > 1) Naively, it could re-bootstrap
> > metadata
> > > > log
> > > > >> > from
> > > > >> > > > the
> > > > >> > > > > > very
> > > > >> > > > > > > > > > >> beginning
> > > > >> > > > > > > > > > >> > to
> > > > >> > > > > > > > > > >> > > > catch up.
> > > > >> > > > > > > > > > >> > > > 2) During that time, it would refrain
> > > itself
> > > > >> from
> > > > >> > > > > > answering
> > > > >> > > > > > > > > > >> > > MetadataRequest
> > > > >> > > > > > > > > > >> > > > from any clients.
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > I am not sure that the "FirstDirtyOffset"
> > > field
> > > > >> > fully
> > > > >> > > > > > > addresses
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > >> > issue.
> > > > >> > > > > > > > > > >> > > Currently, the deletion tombstone is not
> > > > removed
> > > > >> > > > > immediately
> > > > >> > > > > > > > > after a
> > > > >> > > > > > > > > > >> > round
> > > > >> > > > > > > > > > >> > > of cleaning. It's removed after a delay
> in
> > a
> > > > >> > > subsequent
> > > > >> > > > > > round
> > > > >> > > > > > > of
> > > > >> > > > > > > > > > >> > cleaning.
> > > > >> > > > > > > > > > >> > > Consider an example where a key insertion
> > is
> > > at
> > > > >> > offset
> > > > >> > > > 200
> > > > >> > > > > > > and a
> > > > >> > > > > > > > > > >> deletion
> > > > >> > > > > > > > > > >> > > tombstone of the key is at 400.
> Initially,
> > > > >> > > > > FirstDirtyOffset
> > > > >> > > > > > is
> > > > >> > > > > > > > at
> > > > >> > > > > > > > > > >> 300. A
> > > > >> > > > > > > > > > >> > > follower/observer fetches from offset 0
> > and
> > > > >> fetches
> > > > >> > > the
> > > > >> > > > > key
> > > > >> > > > > > > at
> > > > >> > > > > > > > > > offset
> > > > >> > > > > > > > > > >> > 200.
> > > > >> > > > > > > > > > >> > > A few rounds of cleaning happen.
> > > > >> FirstDirtyOffset is
> > > > >> > > at
> > > > >> > > > > 500
> > > > >> > > > > > > and
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > >> > > tombstone at 400 is physically removed.
> The
> > > > >> > > > > > follower/observer
> > > > >> > > > > > > > > > >> continues
> > > > >> > > > > > > > > > >> > the
> > > > >> > > > > > > > > > >> > > fetch, but misses offset 400. It catches
> > all
> > > > the
> > > > >> way
> > > > >> > > to
> > > > >> > > > > > > > > > >> FirstDirtyOffset
> > > > >> > > > > > > > > > >> > > and declares its metadata as ready.
> > However,
> > > > its
> > > > >> > > > metadata
> > > > >> > > > > > > could
> > > > >> > > > > > > > be
> > > > >> > > > > > > > > > >> stale
> > > > >> > > > > > > > > > >> > > since it actually misses the deletion of
> > the
> > > > key.
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > Yeah good question, I should have put
> more
> > > > >> details
> > > > >> > in
> > > > >> > > my
> > > > >> > > > > > > > > explanation
> > > > >> > > > > > > > > > >> :)
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > The idea is that we will adjust the log
> > > > compaction
> > > > >> for
> > > > >> > > > this
> > > > >> > > > > > raft
> > > > >> > > > > > > > > based
> > > > >> > > > > > > > > > >> > metadata log: before more details to be
> > > > explained,
> > > > >> > since
> > > > >> > > > we
> > > > >> > > > > > have
> > > > >> > > > > > > > two
> > > > >> > > > > > > > > > >> types
> > > > >> > > > > > > > > > >> > of "watermarks" here, whereas in Kafka the
> > > > >> watermark
> > > > >> > > > > indicates
> > > > >> > > > > > > > where
> > > > >> > > > > > > > > > >> every
> > > > >> > > > > > > > > > >> > replica have replicated up to and in Raft
> the
> > > > >> > watermark
> > > > >> > > > > > > indicates
> > > > >> > > > > > > > > > where
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > majority of replicas (here only indicating
> > > voters
> > > > >> of
> > > > >> > the
> > > > >> > > > > > quorum,
> > > > >> > > > > > > > not
> > > > >> > > > > > > > > > >> > counting observers) have replicated up to,
> > > let's
> > > > >> call
> > > > >> > > them
> > > > >> > > > > > Kafka
> > > > >> > > > > > > > > > >> watermark
> > > > >> > > > > > > > > > >> > and Raft watermark. For this special log,
> we
> > > > would
> > > > >> > > > maintain
> > > > >> > > > > > both
> > > > >> > > > > > > > > > >> > watermarks.
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > When log compacting on the leader, we would
> > > only
> > > > >> > compact
> > > > >> > > > up
> > > > >> > > > > to
> > > > >> > > > > > > the
> > > > >> > > > > > > > > > Kafka
> > > > >> > > > > > > > > > >> > watermark, i.e. if there is at least one
> > voter
> > > > who
> > > > >> > have
> > > > >> > > > not
> > > > >> > > > > > > > > replicated
> > > > >> > > > > > > > > > >> an
> > > > >> > > > > > > > > > >> > entry, it would not be compacted. The
> > > > >> "dirty-offset"
> > > > >> > is
> > > > >> > > > the
> > > > >> > > > > > > offset
> > > > >> > > > > > > > > > that
> > > > >> > > > > > > > > > >> > we've compacted up to and is communicated
> to
> > > > other
> > > > >> > > voters,
> > > > >> > > > > and
> > > > >> > > > > > > the
> > > > >> > > > > > > > > > other
> > > > >> > > > > > > > > > >> > voters would also compact up to this value
> > ---
> > > > i.e.
> > > > >> > the
> > > > >> > > > > > > difference
> > > > >> > > > > > > > > > here
> > > > >> > > > > > > > > > >> is
> > > > >> > > > > > > > > > >> > that instead of letting each replica doing
> > log
> > > > >> > > compaction
> > > > >> > > > > > > > > > independently,
> > > > >> > > > > > > > > > >> > we'll have the leader to decide upon which
> > > offset
> > > > >> to
> > > > >> > > > compact
> > > > >> > > > > > to,
> > > > >> > > > > > > > and
> > > > >> > > > > > > > > > >> > propagate this value to others to follow,
> in
> > a
> > > > more
> > > > >> > > > > > coordinated
> > > > >> > > > > > > > > > manner.
> > > > >> > > > > > > > > > >> > Also note when there are new voters joining
> > the
> > > > >> quorum
> > > > >> > > who
> > > > >> > > > > has
> > > > >> > > > > > > not
> > > > >> > > > > > > > > > >> > replicated up to the dirty-offset, of
> because
> > > of
> > > > >> other
> > > > >> > > > > issues
> > > > >> > > > > > > they
> > > > >> > > > > > > > > > >> > truncated their logs to below the
> > dirty-offset,
> > > > >> they'd
> > > > >> > > > have
> > > > >> > > > > to
> > > > >> > > > > > > > > > >> re-bootstrap
> > > > >> > > > > > > > > > >> > from the beginning, and during this period
> of
> > > > time
> > > > >> the
> > > > >> > > > > leader
> > > > >> > > > > > > > > learned
> > > > >> > > > > > > > > > >> about
> > > > >> > > > > > > > > > >> > this lagging voter would not advance the
> > > > watermark
> > > > >> > (also
> > > > >> > > > it
> > > > >> > > > > > > would
> > > > >> > > > > > > > > not
> > > > >> > > > > > > > > > >> > decrement it), and hence not compacting
> > either,
> > > > >> until
> > > > >> > > the
> > > > >> > > > > > > voter(s)
> > > > >> > > > > > > > > has
> > > > >> > > > > > > > > > >> > caught up to that dirty-offset.
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > So back to your example above, before the
> > > > bootstrap
> > > > >> > > voter
> > > > >> > > > > gets
> > > > >> > > > > > > to
> > > > >> > > > > > > > > 300
> > > > >> > > > > > > > > > no
> > > > >> > > > > > > > > > >> > log compaction would happen on the leader;
> > and
> > > > >> until
> > > > >> > > later
> > > > >> > > > > > when
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > >> voter
> > > > >> > > > > > > > > > >> > have got to beyond 400 and hence replicated
> > > that
> > > > >> > > > tombstone,
> > > > >> > > > > > the
> > > > >> > > > > > > > log
> > > > >> > > > > > > > > > >> > compaction would possibly get to that
> > tombstone
> > > > and
> > > > >> > > remove
> > > > >> > > > > it.
> > > > >> > > > > > > Say
> > > > >> > > > > > > > > > >> later it
> > > > >> > > > > > > > > > >> > the leader's log compaction reaches 500, it
> > can
> > > > >> send
> > > > >> > > this
> > > > >> > > > > back
> > > > >> > > > > > > to
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > >> voter
> > > > >> > > > > > > > > > >> > who can then also compact locally up to
> 500.
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > > > > 105. Quorum State: In addition to
> > > VotedId,
> > > > >> do we
> > > > >> > > > need
> > > > >> > > > > > the
> > > > >> > > > > > > > > epoch
> > > > >> > > > > > > > > > >> > > > > corresponding to VotedId? Over time,
> > the
> > > > same
> > > > >> > > broker
> > > > >> > > > > Id
> > > > >> > > > > > > > could
> > > > >> > > > > > > > > be
> > > > >> > > > > > > > > > >> > voted
> > > > >> > > > > > > > > > >> > > in
> > > > >> > > > > > > > > > >> > > > > different generations with different
> > > epoch.
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > > Hmm, this is a good point. Originally I
> > > think
> > > > >> the
> > > > >> > > > > > > > "LeaderEpoch"
> > > > >> > > > > > > > > > >> field
> > > > >> > > > > > > > > > >> > in
> > > > >> > > > > > > > > > >> > > > that file is corresponding to the
> "latest
> > > > known
> > > > >> > > leader
> > > > >> > > > > > > epoch",
> > > > >> > > > > > > > > not
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > > > "current leader epoch". For example, if
> > the
> > > > >> > current
> > > > >> > > > > epoch
> > > > >> > > > > > is
> > > > >> > > > > > > > N,
> > > > >> > > > > > > > > > and
> > > > >> > > > > > > > > > >> > then
> > > > >> > > > > > > > > > >> > > a
> > > > >> > > > > > > > > > >> > > > vote-request with epoch N+1 is received
> > and
> > > > the
> > > > >> > > voter
> > > > >> > > > > > > granted
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > >> vote
> > > > >> > > > > > > > > > >> > > for
> > > > >> > > > > > > > > > >> > > > it, then it means for this voter it
> knows
> > > the
> > > > >> > > "latest
> > > > >> > > > > > epoch"
> > > > >> > > > > > > > is
> > > > >> > > > > > > > > N
> > > > >> > > > > > > > > > +
> > > > >> > > > > > > > > > >> 1
> > > > >> > > > > > > > > > >> > > > although it is unknown if that sending
> > > > >> candidate
> > > > >> > > will
> > > > >> > > > > > indeed
> > > > >> > > > > > > > > > become
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > > new
> > > > >> > > > > > > > > > >> > > > leader (which would only be notified
> via
> > > > >> > > begin-quorum
> > > > >> > > > > > > > request).
> > > > >> > > > > > > > > > >> > However,
> > > > >> > > > > > > > > > >> > > > when persisting the quorum state, we
> > would
> > > > >> encode
> > > > >> > > > > > > leader-epoch
> > > > >> > > > > > > > > to
> > > > >> > > > > > > > > > >> N+1,
> > > > >> > > > > > > > > > >> > > > while the leaderId to be the older
> > leader.
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > But now thinking about this a bit
> more, I
> > > > feel
> > > > >> we
> > > > >> > > > should
> > > > >> > > > > > use
> > > > >> > > > > > > > two
> > > > >> > > > > > > > > > >> > separate
> > > > >> > > > > > > > > > >> > > > epochs, one for the "lates known" and
> one
> > > for
> > > > >> the
> > > > >> > > > > > "current"
> > > > >> > > > > > > to
> > > > >> > > > > > > > > > pair
> > > > >> > > > > > > > > > >> > with
> > > > >> > > > > > > > > > >> > > > the leaderId. I will update the wiki
> > page.
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > Hmm, it's kind of weird to bump up the
> > leader
> > > > >> epoch
> > > > >> > > > before
> > > > >> > > > > > the
> > > > >> > > > > > > > new
> > > > >> > > > > > > > > > >> leader
> > > > >> > > > > > > > > > >> > > is actually elected, right.
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in
> the
> > > > >> > > > > > FetchQuorumRecords
> > > > >> > > > > > > > API
> > > > >> > > > > > > > > to
> > > > >> > > > > > > > > > >> > > indicate
> > > > >> > > > > > > > > > >> > > > > that the follower has fetched from an
> > > > invalid
> > > > >> > > offset
> > > > >> > > > > and
> > > > >> > > > > > > > > should
> > > > >> > > > > > > > > > >> > > truncate
> > > > >> > > > > > > > > > >> > > > to
> > > > >> > > > > > > > > > >> > > > > the offset/epoch indicated in the
> > > > response."
> > > > >> > > > Observers
> > > > >> > > > > > > can't
> > > > >> > > > > > > > > > >> truncate
> > > > >> > > > > > > > > > >> > > > their
> > > > >> > > > > > > > > > >> > > > > logs. What should they do with
> > > > >> > > OFFSET_OUT_OF_RANGE?
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > > I'm not sure if I understand your
> > question?
> > > > >> > > Observers
> > > > >> > > > > > should
> > > > >> > > > > > > > > still
> > > > >> > > > > > > > > > >> be
> > > > >> > > > > > > > > > >> > > able
> > > > >> > > > > > > > > > >> > > > to truncate their logs as well.
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > Hmm, I thought only the quorum nodes have
> > > local
> > > > >> logs
> > > > >> > > and
> > > > >> > > > > > > > observers
> > > > >> > > > > > > > > > >> don't?
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > > 107. "The leader will continue sending
> > > > >> > > > BeginQuorumEpoch
> > > > >> > > > > to
> > > > >> > > > > > > > each
> > > > >> > > > > > > > > > >> known
> > > > >> > > > > > > > > > >> > > > voter
> > > > >> > > > > > > > > > >> > > > > until it has received its
> endorsement."
> > > If
> > > > a
> > > > >> > voter
> > > > >> > > > is
> > > > >> > > > > > down
> > > > >> > > > > > > > > for a
> > > > >> > > > > > > > > > >> long
> > > > >> > > > > > > > > > >> > > > time,
> > > > >> > > > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add
> > > > >> > unnecessary
> > > > >> > > > > > > overhead.
> > > > >> > > > > > > > > > >> > Similarly,
> > > > >> > > > > > > > > > >> > > > if a
> > > > >> > > > > > > > > > >> > > > > follower stops sending
> > > FetchQuorumRecords,
> > > > >> does
> > > > >> > > the
> > > > >> > > > > > leader
> > > > >> > > > > > > > > keep
> > > > >> > > > > > > > > > >> > sending
> > > > >> > > > > > > > > > >> > > > > BeginQuorumEpoch?
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a
> > good
> > > > >> point.
> > > > >> > > The
> > > > >> > > > > > > > > > >> > begin-quorum-epoch
> > > > >> > > > > > > > > > >> > > > request is for voters to quickly get
> the
> > > new
> > > > >> > leader
> > > > >> > > > > > > > information;
> > > > >> > > > > > > > > > >> > however
> > > > >> > > > > > > > > > >> > > > even if they do not get them they can
> > still
> > > > >> > > eventually
> > > > >> > > > > > learn
> > > > >> > > > > > > > > about
> > > > >> > > > > > > > > > >> that
> > > > >> > > > > > > > > > >> > > > from others via gossiping FindQuorum. I
> > > think
> > > > >> we
> > > > >> > can
> > > > >> > > > > > adjust
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > >> logic
> > > > >> > > > > > > > > > >> > to
> > > > >> > > > > > > > > > >> > > > e.g. exponential back-off or with a
> > limited
> > > > >> > > > num.retries.
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > Regarding FetchQuorumRecords: if the
> > > follower
> > > > >> > sends
> > > > >> > > > > > > > > > >> FetchQuorumRecords
> > > > >> > > > > > > > > > >> > > > already, it means that follower already
> > > knows
> > > > >> that
> > > > >> > > the
> > > > >> > > > > > > broker
> > > > >> > > > > > > > is
> > > > >> > > > > > > > > > the
> > > > >> > > > > > > > > > >> > > > leader, and hence we can stop retrying
> > > > >> > > > BeginQuorumEpoch;
> > > > >> > > > > > > > however
> > > > >> > > > > > > > > > it
> > > > >> > > > > > > > > > >> is
> > > > >> > > > > > > > > > >> > > > possible that after a follower sends
> > > > >> > > > FetchQuorumRecords
> > > > >> > > > > > > > already,
> > > > >> > > > > > > > > > >> > suddenly
> > > > >> > > > > > > > > > >> > > > it stops send it (possibly because it
> > > learned
> > > > >> > about
> > > > >> > > a
> > > > >> > > > > > higher
> > > > >> > > > > > > > > epoch
> > > > >> > > > > > > > > > >> > > leader),
> > > > >> > > > > > > > > > >> > > > and hence this broker may be a "zombie"
> > > > leader
> > > > >> and
> > > > >> > > we
> > > > >> > > > > > > propose
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > > use
> > > > >> > > > > > > > > > >> > the
> > > > >> > > > > > > > > > >> > > > fetch.timeout to let the leader to try
> to
> > > > >> verify
> > > > >> > if
> > > > >> > > it
> > > > >> > > > > has
> > > > >> > > > > > > > > already
> > > > >> > > > > > > > > > >> been
> > > > >> > > > > > > > > > >> > > > stale.
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > It just seems that we should handle these
> > two
> > > > >> cases
> > > > >> > > in a
> > > > >> > > > > > > > > consistent
> > > > >> > > > > > > > > > >> way?
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > Yes I agree, on the leader's side, the
> > > > >> > > > FetchQuorumRecords
> > > > >> > > > > > > from a
> > > > >> > > > > > > > > > >> follower
> > > > >> > > > > > > > > > >> > could mean that we no longer needs to send
> > > > >> > > > BeginQuorumEpoch
> > > > >> > > > > > > > anymore
> > > > >> > > > > > > > > > ---
> > > > >> > > > > > > > > > >> and
> > > > >> > > > > > > > > > >> > it is already part of our current
> > > implementations
> > > > >> in
> > > > >> > > > > > > > > > >> >
> > > > >> > > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > > Thanks,
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > Jun
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > > > Jun
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM
> > Guozhang
> > > > >> Wang <
> > > > >> > > > > > > > > > wangguoz@gmail.com
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > > > wrote:
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > > > > Hello Leonard,
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > Thanks for your comments, I'm
> relying
> > > in
> > > > >> line
> > > > >> > > > below:
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM
> Wang
> > > > >> (Leonard)
> > > > >> > > Ge
> > > > >> > > > <
> > > > >> > > > > > > > > > >> > wge@confluent.io>
> > > > >> > > > > > > > > > >> > > > > > wrote:
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > > Hi Kafka developers,
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > It's great to see this proposal
> and
> > > it
> > > > >> took
> > > > >> > me
> > > > >> > > > > some
> > > > >> > > > > > > time
> > > > >> > > > > > > > > to
> > > > >> > > > > > > > > > >> > finish
> > > > >> > > > > > > > > > >> > > > > > reading
> > > > >> > > > > > > > > > >> > > > > > > it.
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > And I have the following
> questions
> > > > about
> > > > >> the
> > > > >> > > > > > Proposal:
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > >    - How do we plan to test this
> > > design
> > > > >> to
> > > > >> > > > ensure
> > > > >> > > > > > its
> > > > >> > > > > > > > > > >> > correctness?
> > > > >> > > > > > > > > > >> > > Or
> > > > >> > > > > > > > > > >> > > > > > more
> > > > >> > > > > > > > > > >> > > > > > >    broadly, how do we ensure that
> > our
> > > > new
> > > > >> > > ‘pull’
> > > > >> > > > > > based
> > > > >> > > > > > > > > model
> > > > >> > > > > > > > > > >> is
> > > > >> > > > > > > > > > >> > > > > > functional
> > > > >> > > > > > > > > > >> > > > > > > and
> > > > >> > > > > > > > > > >> > > > > > >    correct given that it is
> > different
> > > > >> from
> > > > >> > the
> > > > >> > > > > > > original
> > > > >> > > > > > > > > RAFT
> > > > >> > > > > > > > > > >> > > > > > implementation
> > > > >> > > > > > > > > > >> > > > > > >    which has formal proof of
> > > > correctness?
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > We have two planned verifications
> on
> > > the
> > > > >> > > > correctness
> > > > >> > > > > > and
> > > > >> > > > > > > > > > >> liveness
> > > > >> > > > > > > > > > >> > of
> > > > >> > > > > > > > > > >> > > > the
> > > > >> > > > > > > > > > >> > > > > > design. One is via model
> verification
> > > > >> (TLA+)
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > https://github.com/guozhangwang/kafka-specification
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > Another is via the concurrent
> > > simulation
> > > > >> tests
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > >    - Have we considered any
> sensible
> > > > >> defaults
> > > > >> > > for
> > > > >> > > > > the
> > > > >> > > > > > > > > > >> > configuration,
> > > > >> > > > > > > > > > >> > > > i.e.
> > > > >> > > > > > > > > > >> > > > > > >    all the election timeout,
> fetch
> > > time
> > > > >> out,
> > > > >> > > > etc.?
> > > > >> > > > > > Or
> > > > >> > > > > > > we
> > > > >> > > > > > > > > > want
> > > > >> > > > > > > > > > >> to
> > > > >> > > > > > > > > > >> > > > leave
> > > > >> > > > > > > > > > >> > > > > > > this to
> > > > >> > > > > > > > > > >> > > > > > >    a later stage when we do the
> > > > >> performance
> > > > >> > > > > testing,
> > > > >> > > > > > > > etc.
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > This is a good question, the reason
> > we
> > > > did
> > > > >> not
> > > > >> > > set
> > > > >> > > > > any
> > > > >> > > > > > > > > default
> > > > >> > > > > > > > > > >> > values
> > > > >> > > > > > > > > > >> > > > for
> > > > >> > > > > > > > > > >> > > > > > the timeout configurations is that
> we
> > > > >> think it
> > > > >> > > may
> > > > >> > > > > > take
> > > > >> > > > > > > > some
> > > > >> > > > > > > > > > >> > > > benchmarking
> > > > >> > > > > > > > > > >> > > > > > experiments to get these defaults
> > > right.
> > > > >> Some
> > > > >> > > > > > high-level
> > > > >> > > > > > > > > > >> principles
> > > > >> > > > > > > > > > >> > > to
> > > > >> > > > > > > > > > >> > > > > > consider: 1) the fetch.timeout
> should
> > > be
> > > > >> > around
> > > > >> > > > the
> > > > >> > > > > > same
> > > > >> > > > > > > > > scale
> > > > >> > > > > > > > > > >> with
> > > > >> > > > > > > > > > >> > > zk
> > > > >> > > > > > > > > > >> > > > > > session timeout, which is now 18
> > > seconds
> > > > by
> > > > >> > > > default
> > > > >> > > > > --
> > > > >> > > > > > > in
> > > > >> > > > > > > > > > >> practice
> > > > >> > > > > > > > > > >> > > > we've
> > > > >> > > > > > > > > > >> > > > > > seen unstable networks having more
> > than
> > > > 10
> > > > >> > secs
> > > > >> > > of
> > > > >> > > > > > > > transient
> > > > >> > > > > > > > > > >> > > > > connectivity,
> > > > >> > > > > > > > > > >> > > > > > 2) the election.timeout, however,
> > > should
> > > > be
> > > > >> > > > smaller
> > > > >> > > > > > than
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > >> fetch
> > > > >> > > > > > > > > > >> > > > > timeout
> > > > >> > > > > > > > > > >> > > > > > as is also suggested as a practical
> > > > >> > optimization
> > > > >> > > > in
> > > > >> > > > > > > > > > literature:
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > >
> > > > >> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > Some more discussions can be found
> > > here:
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > >
> > > > >> > > >
> > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > >    - Have we considered
> > piggybacking
> > > > >> > > > > > > `BeginQuorumEpoch`
> > > > >> > > > > > > > > with
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > `
> > > > >> > > > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might
> be
> > > > >> missing
> > > > >> > > > > something
> > > > >> > > > > > > > > obvious
> > > > >> > > > > > > > > > >> but
> > > > >> > > > > > > > > > >> > I
> > > > >> > > > > > > > > > >> > > am
> > > > >> > > > > > > > > > >> > > > > > just
> > > > >> > > > > > > > > > >> > > > > > >    wondering why don’t we just
> use
> > > the
> > > > >> > > > > `FindQuorum`
> > > > >> > > > > > > and
> > > > >> > > > > > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > >> > > > > > > > > > >> > > > > > >    APIs and remove the
> > > > `BeginQuorumEpoch`
> > > > >> > API?
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is
> > sent
> > > > from
> > > > >> > > leader
> > > > >> > > > > ->
> > > > >> > > > > > > > other
> > > > >> > > > > > > > > > >> voter
> > > > >> > > > > > > > > > >> > > > > > followers, while FindQuorum / Fetch
> > are
> > > > >> sent
> > > > >> > > from
> > > > >> > > > > > > follower
> > > > >> > > > > > > > > to
> > > > >> > > > > > > > > > >> > leader.
> > > > >> > > > > > > > > > >> > > > > > Arguably one can eventually realize
> > the
> > > > new
> > > > >> > > leader
> > > > >> > > > > and
> > > > >> > > > > > > > epoch
> > > > >> > > > > > > > > > via
> > > > >> > > > > > > > > > >> > > > > gossiping
> > > > >> > > > > > > > > > >> > > > > > FindQuorum, but that could in
> > practice
> > > > >> > require a
> > > > >> > > > > long
> > > > >> > > > > > > > delay.
> > > > >> > > > > > > > > > >> > Having a
> > > > >> > > > > > > > > > >> > > > > > leader -> other voters request
> helps
> > > the
> > > > >> new
> > > > >> > > > leader
> > > > >> > > > > > > epoch
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > > be
> > > > >> > > > > > > > > > >> > > > > propagated
> > > > >> > > > > > > > > > >> > > > > > faster under a pull model.
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > >    - And about the
> > > `FetchQuorumRecords`
> > > > >> > > response
> > > > >> > > > > > > schema,
> > > > >> > > > > > > > > in
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > > > > `Records`
> > > > >> > > > > > > > > > >> > > > > > >    field of the response, is it
> > just
> > > > one
> > > > >> > > record
> > > > >> > > > or
> > > > >> > > > > > all
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > >> > records
> > > > >> > > > > > > > > > >> > > > > > starting
> > > > >> > > > > > > > > > >> > > > > > >    from the FetchOffset? It
> seems a
> > > lot
> > > > >> more
> > > > >> > > > > > efficient
> > > > >> > > > > > > > if
> > > > >> > > > > > > > > we
> > > > >> > > > > > > > > > >> sent
> > > > >> > > > > > > > > > >> > > all
> > > > >> > > > > > > > > > >> > > > > the
> > > > >> > > > > > > > > > >> > > > > > >    records during the
> bootstrapping
> > > of
> > > > >> the
> > > > >> > > > > brokers.
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > Yes the fetching is batched:
> > > FetchOffset
> > > > is
> > > > >> > just
> > > > >> > > > the
> > > > >> > > > > > > > > starting
> > > > >> > > > > > > > > > >> > offset
> > > > >> > > > > > > > > > >> > > of
> > > > >> > > > > > > > > > >> > > > > the
> > > > >> > > > > > > > > > >> > > > > > batch of records.
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > >    - Regarding the disruptive
> > broker
> > > > >> issues,
> > > > >> > > > does
> > > > >> > > > > > our
> > > > >> > > > > > > > pull
> > > > >> > > > > > > > > > >> based
> > > > >> > > > > > > > > > >> > > > model
> > > > >> > > > > > > > > > >> > > > > > >    suffer from it? If so, have we
> > > > >> considered
> > > > >> > > the
> > > > >> > > > > > > > Pre-Vote
> > > > >> > > > > > > > > > >> stage?
> > > > >> > > > > > > > > > >> > If
> > > > >> > > > > > > > > > >> > > > > not,
> > > > >> > > > > > > > > > >> > > > > > > why?
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > The disruptive broker is stated in
> > the
> > > > >> > original
> > > > >> > > > Raft
> > > > >> > > > > > > paper
> > > > >> > > > > > > > > > >> which is
> > > > >> > > > > > > > > > >> > > the
> > > > >> > > > > > > > > > >> > > > > > result of the push model design.
> Our
> > > > >> analysis
> > > > >> > > > showed
> > > > >> > > > > > > that
> > > > >> > > > > > > > > with
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > > pull
> > > > >> > > > > > > > > > >> > > > > > model it is no longer an issue.
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > > Thanks a lot for putting this up,
> > > and I
> > > > >> hope
> > > > >> > > > that
> > > > >> > > > > my
> > > > >> > > > > > > > > > questions
> > > > >> > > > > > > > > > >> > can
> > > > >> > > > > > > > > > >> > > be
> > > > >> > > > > > > > > > >> > > > > of
> > > > >> > > > > > > > > > >> > > > > > > some value to make this KIP
> better.
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > Hope to hear from you soon!
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > Best wishes,
> > > > >> > > > > > > > > > >> > > > > > > Leonard
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM
> > Colin
> > > > >> > McCabe <
> > > > >> > > > > > > > > > >> cmccabe@apache.org
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> > > > > wrote:
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > Hi Jason,
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > It's amazing to see this coming
> > > > >> together
> > > > >> > :)
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > I haven't had a chance to read
> in
> > > > >> detail,
> > > > >> > > but
> > > > >> > > > I
> > > > >> > > > > > read
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > >> > outline
> > > > >> > > > > > > > > > >> > > > and
> > > > >> > > > > > > > > > >> > > > > a
> > > > >> > > > > > > > > > >> > > > > > > few
> > > > >> > > > > > > > > > >> > > > > > > > things jumped out at me.
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > First, for every epoch that is
> 32
> > > > bits
> > > > >> > > rather
> > > > >> > > > > than
> > > > >> > > > > > > > 64, I
> > > > >> > > > > > > > > > >> sort
> > > > >> > > > > > > > > > >> > of
> > > > >> > > > > > > > > > >> > > > > wonder
> > > > >> > > > > > > > > > >> > > > > > > if
> > > > >> > > > > > > > > > >> > > > > > > > that's a good long-term choice.
> > I
> > > > keep
> > > > >> > > > reading
> > > > >> > > > > > > about
> > > > >> > > > > > > > > > stuff
> > > > >> > > > > > > > > > >> > like
> > > > >> > > > > > > > > > >> > > > > this:
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > >> > > > > > > > .
> > > > >> > > > > > > > > > >> > > Obviously,
> > > > >> > > > > > > > > > >> > > > > > that
> > > > >> > > > > > > > > > >> > > > > > > > JIRA is about zxid, which
> > > increments
> > > > >> much
> > > > >> > > > faster
> > > > >> > > > > > > than
> > > > >> > > > > > > > we
> > > > >> > > > > > > > > > >> expect
> > > > >> > > > > > > > > > >> > > > these
> > > > >> > > > > > > > > > >> > > > > > > > leader epochs to, but it would
> > > still
> > > > be
> > > > >> > good
> > > > >> > > > to
> > > > >> > > > > > see
> > > > >> > > > > > > > some
> > > > >> > > > > > > > > > >> rough
> > > > >> > > > > > > > > > >> > > > > > > calculations
> > > > >> > > > > > > > > > >> > > > > > > > about how long 32 bits (or
> > really,
> > > 31
> > > > >> > bits)
> > > > >> > > > will
> > > > >> > > > > > > last
> > > > >> > > > > > > > us
> > > > >> > > > > > > > > > in
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > > > cases
> > > > >> > > > > > > > > > >> > > > > > > where
> > > > >> > > > > > > > > > >> > > > > > > > we're using it, and what the
> > space
> > > > >> savings
> > > > >> > > > we're
> > > > >> > > > > > > > getting
> > > > >> > > > > > > > > > >> really
> > > > >> > > > > > > > > > >> > > is.
> > > > >> > > > > > > > > > >> > > > > It
> > > > >> > > > > > > > > > >> > > > > > > > seems like in most cases the
> > > tradeoff
> > > > >> may
> > > > >> > > not
> > > > >> > > > be
> > > > >> > > > > > > worth
> > > > >> > > > > > > > > it?
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > Another thing I've been
> thinking
> > > > about
> > > > >> is
> > > > >> > > how
> > > > >> > > > we
> > > > >> > > > > > do
> > > > >> > > > > > > > > > >> > > > bootstrapping.  I
> > > > >> > > > > > > > > > >> > > > > > > > would prefer to be in a world
> > where
> > > > >> > > > formatting a
> > > > >> > > > > > new
> > > > >> > > > > > > > > Kafka
> > > > >> > > > > > > > > > >> node
> > > > >> > > > > > > > > > >> > > > was a
> > > > >> > > > > > > > > > >> > > > > > > first
> > > > >> > > > > > > > > > >> > > > > > > > class operation explicitly
> > > initiated
> > > > by
> > > > >> > the
> > > > >> > > > > admin,
> > > > >> > > > > > > > > rather
> > > > >> > > > > > > > > > >> than
> > > > >> > > > > > > > > > >> > > > > > something
> > > > >> > > > > > > > > > >> > > > > > > > that happened implicitly when
> you
> > > > >> started
> > > > >> > up
> > > > >> > > > the
> > > > >> > > > > > > > broker
> > > > >> > > > > > > > > > and
> > > > >> > > > > > > > > > >> > > things
> > > > >> > > > > > > > > > >> > > > > > > "looked
> > > > >> > > > > > > > > > >> > > > > > > > blank."
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > The first problem is that
> things
> > > can
> > > > >> "look
> > > > >> > > > > blank"
> > > > >> > > > > > > > > > >> accidentally
> > > > >> > > > > > > > > > >> > if
> > > > >> > > > > > > > > > >> > > > the
> > > > >> > > > > > > > > > >> > > > > > > > storage system is having a bad
> > day.
> > > > >> > Clearly
> > > > >> > > > in
> > > > >> > > > > > the
> > > > >> > > > > > > > > > non-Raft
> > > > >> > > > > > > > > > >> > > world,
> > > > >> > > > > > > > > > >> > > > > > this
> > > > >> > > > > > > > > > >> > > > > > > > leads to data loss if the
> broker
> > > that
> > > > >> is
> > > > >> > > > > > (re)started
> > > > >> > > > > > > > > this
> > > > >> > > > > > > > > > >> way
> > > > >> > > > > > > > > > >> > was
> > > > >> > > > > > > > > > >> > > > the
> > > > >> > > > > > > > > > >> > > > > > > > leader for some partitions.
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > The second problem is that we
> > have
> > > a
> > > > >> bit
> > > > >> > of
> > > > >> > > a
> > > > >> > > > > > > chicken
> > > > >> > > > > > > > > and
> > > > >> > > > > > > > > > >> egg
> > > > >> > > > > > > > > > >> > > > problem
> > > > >> > > > > > > > > > >> > > > > > > with
> > > > >> > > > > > > > > > >> > > > > > > > certain configuration keys.
> For
> > > > >> example,
> > > > >> > > > maybe
> > > > >> > > > > > you
> > > > >> > > > > > > > want
> > > > >> > > > > > > > > > to
> > > > >> > > > > > > > > > >> > > > configure
> > > > >> > > > > > > > > > >> > > > > > > some
> > > > >> > > > > > > > > > >> > > > > > > > connection security settings in
> > > your
> > > > >> > > cluster,
> > > > >> > > > > but
> > > > >> > > > > > > you
> > > > >> > > > > > > > > > don't
> > > > >> > > > > > > > > > >> > want
> > > > >> > > > > > > > > > >> > > > them
> > > > >> > > > > > > > > > >> > > > > > to
> > > > >> > > > > > > > > > >> > > > > > > > ever be stored in a plaintext
> > > config
> > > > >> file.
> > > > >> > > > (For
> > > > >> > > > > > > > > example,
> > > > >> > > > > > > > > > >> SCRAM
> > > > >> > > > > > > > > > >> > > > > > > passwords,
> > > > >> > > > > > > > > > >> > > > > > > > etc.)  You could use a broker
> API
> > > to
> > > > >> set
> > > > >> > the
> > > > >> > > > > > > > > > configuration,
> > > > >> > > > > > > > > > >> but
> > > > >> > > > > > > > > > >> > > > that
> > > > >> > > > > > > > > > >> > > > > > > brings
> > > > >> > > > > > > > > > >> > > > > > > > up the chicken and egg problem.
> > > The
> > > > >> > broker
> > > > >> > > > > needs
> > > > >> > > > > > to
> > > > >> > > > > > > > be
> > > > >> > > > > > > > > > >> > > configured
> > > > >> > > > > > > > > > >> > > > to
> > > > >> > > > > > > > > > >> > > > > > > know
> > > > >> > > > > > > > > > >> > > > > > > > how to talk to you, but you
> need
> > to
> > > > >> > > configure
> > > > >> > > > it
> > > > >> > > > > > > > before
> > > > >> > > > > > > > > > you
> > > > >> > > > > > > > > > >> can
> > > > >> > > > > > > > > > >> > > > talk
> > > > >> > > > > > > > > > >> > > > > to
> > > > >> > > > > > > > > > >> > > > > > > > it.  Using an external secret
> > > manager
> > > > >> like
> > > > >> > > > Vault
> > > > >> > > > > > is
> > > > >> > > > > > > > one
> > > > >> > > > > > > > > > way
> > > > >> > > > > > > > > > >> to
> > > > >> > > > > > > > > > >> > > > solve
> > > > >> > > > > > > > > > >> > > > > > > this,
> > > > >> > > > > > > > > > >> > > > > > > > but not everyone uses an
> external
> > > > >> secret
> > > > >> > > > > manager.
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > quorum.voters seems like a
> > similar
> > > > >> > > > configuration
> > > > >> > > > > > > key.
> > > > >> > > > > > > > > In
> > > > >> > > > > > > > > > >> the
> > > > >> > > > > > > > > > >> > > > current
> > > > >> > > > > > > > > > >> > > > > > > KIP,
> > > > >> > > > > > > > > > >> > > > > > > > this is only read if there is
> no
> > > > other
> > > > >> > > > > > configuration
> > > > >> > > > > > > > > > >> specifying
> > > > >> > > > > > > > > > >> > > the
> > > > >> > > > > > > > > > >> > > > > > > quorum
> > > > >> > > > > > > > > > >> > > > > > > > voter set.  If we had a
> > kafka.mkfs
> > > > >> > command,
> > > > >> > > we
> > > > >> > > > > > > > wouldn't
> > > > >> > > > > > > > > > need
> > > > >> > > > > > > > > > >> > this
> > > > >> > > > > > > > > > >> > > > key
> > > > >> > > > > > > > > > >> > > > > > > > because we could assume that
> > there
> > > > was
> > > > >> > > always
> > > > >> > > > > > quorum
> > > > >> > > > > > > > > > >> > information
> > > > >> > > > > > > > > > >> > > > > stored
> > > > >> > > > > > > > > > >> > > > > > > > locally.
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > best,
> > > > >> > > > > > > > > > >> > > > > > > > Colin
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44,
> > > Jason
> > > > >> > > > Gustafson
> > > > >> > > > > > > wrote:
> > > > >> > > > > > > > > > >> > > > > > > > > Hi All,
> > > > >> > > > > > > > > > >> > > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > > I'd like to start a
> discussion
> > on
> > > > >> > KIP-595:
> > > > >> > > > > > > > > > >> > > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > >> > > > > > > > > > >> > > > > > > > .
> > > > >> > > > > > > > > > >> > > > > > > > > This proposal specifies a
> Raft
> > > > >> protocol
> > > > >> > to
> > > > >> > > > > > > > ultimately
> > > > >> > > > > > > > > > >> replace
> > > > >> > > > > > > > > > >> > > > > > Zookeeper
> > > > >> > > > > > > > > > >> > > > > > > > > as
> > > > >> > > > > > > > > > >> > > > > > > > > documented in KIP-500. Please
> > > take
> > > > a
> > > > >> > look
> > > > >> > > > and
> > > > >> > > > > > > share
> > > > >> > > > > > > > > your
> > > > >> > > > > > > > > > >> > > > thoughts.
> > > > >> > > > > > > > > > >> > > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > > A few minor notes to set the
> > > stage
> > > > a
> > > > >> > > little
> > > > >> > > > > bit:
> > > > >> > > > > > > > > > >> > > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > > - This KIP does not specify
> the
> > > > >> > structure
> > > > >> > > of
> > > > >> > > > > the
> > > > >> > > > > > > > > > messages
> > > > >> > > > > > > > > > >> > used
> > > > >> > > > > > > > > > >> > > to
> > > > >> > > > > > > > > > >> > > > > > > > represent
> > > > >> > > > > > > > > > >> > > > > > > > > metadata in Kafka, nor does
> it
> > > > >> specify
> > > > >> > the
> > > > >> > > > > > > internal
> > > > >> > > > > > > > > API
> > > > >> > > > > > > > > > >> that
> > > > >> > > > > > > > > > >> > > will
> > > > >> > > > > > > > > > >> > > > > be
> > > > >> > > > > > > > > > >> > > > > > > used
> > > > >> > > > > > > > > > >> > > > > > > > > by the controller. Expect
> these
> > > to
> > > > >> come
> > > > >> > in
> > > > >> > > > > later
> > > > >> > > > > > > > > > >> proposals.
> > > > >> > > > > > > > > > >> > > Here
> > > > >> > > > > > > > > > >> > > > we
> > > > >> > > > > > > > > > >> > > > > > are
> > > > >> > > > > > > > > > >> > > > > > > > > primarily concerned with the
> > > > >> replication
> > > > >> > > > > > protocol
> > > > >> > > > > > > > and
> > > > >> > > > > > > > > > >> basic
> > > > >> > > > > > > > > > >> > > > > > operational
> > > > >> > > > > > > > > > >> > > > > > > > > mechanics.
> > > > >> > > > > > > > > > >> > > > > > > > > - We expect many details to
> > > change
> > > > >> as we
> > > > >> > > get
> > > > >> > > > > > > closer
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > > >> > > > integration
> > > > >> > > > > > > > > > >> > > > > > with
> > > > >> > > > > > > > > > >> > > > > > > > > the controller. Any changes
> we
> > > make
> > > > >> will
> > > > >> > > be
> > > > >> > > > > made
> > > > >> > > > > > > > > either
> > > > >> > > > > > > > > > as
> > > > >> > > > > > > > > > >> > > > > amendments
> > > > >> > > > > > > > > > >> > > > > > > to
> > > > >> > > > > > > > > > >> > > > > > > > > this KIP or, in the case of
> > > larger
> > > > >> > > changes,
> > > > >> > > > as
> > > > >> > > > > > new
> > > > >> > > > > > > > > > >> proposals.
> > > > >> > > > > > > > > > >> > > > > > > > > - We have a prototype
> > > > implementation
> > > > >> > > which I
> > > > >> > > > > > will
> > > > >> > > > > > > > put
> > > > >> > > > > > > > > > >> online
> > > > >> > > > > > > > > > >> > > > within
> > > > >> > > > > > > > > > >> > > > > > the
> > > > >> > > > > > > > > > >> > > > > > > > > next week which may help in
> > > > >> > understanding
> > > > >> > > > some
> > > > >> > > > > > > > > details.
> > > > >> > > > > > > > > > It
> > > > >> > > > > > > > > > >> > has
> > > > >> > > > > > > > > > >> > > > > > > diverged a
> > > > >> > > > > > > > > > >> > > > > > > > > little bit from our proposal,
> > so
> > > I
> > > > am
> > > > >> > > > taking a
> > > > >> > > > > > > > little
> > > > >> > > > > > > > > > >> time to
> > > > >> > > > > > > > > > >> > > > bring
> > > > >> > > > > > > > > > >> > > > > > it
> > > > >> > > > > > > > > > >> > > > > > > in
> > > > >> > > > > > > > > > >> > > > > > > > > line. I'll post an update to
> > this
> > > > >> thread
> > > > >> > > > when
> > > > >> > > > > it
> > > > >> > > > > > > is
> > > > >> > > > > > > > > > >> available
> > > > >> > > > > > > > > > >> > > for
> > > > >> > > > > > > > > > >> > > > > > > review.
> > > > >> > > > > > > > > > >> > > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > > Finally, I want to mention
> that
> > > > this
> > > > >> > > > proposal
> > > > >> > > > > > was
> > > > >> > > > > > > > > > drafted
> > > > >> > > > > > > > > > >> by
> > > > >> > > > > > > > > > >> > > > > myself,
> > > > >> > > > > > > > > > >> > > > > > > > Boyang
> > > > >> > > > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > >> > > > > > > > > > >> > > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > > > Thanks,
> > > > >> > > > > > > > > > >> > > > > > > > > Jason
> > > > >> > > > > > > > > > >> > > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > >
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > > > --
> > > > >> > > > > > > > > > >> > > > > > > Leonard Ge
> > > > >> > > > > > > > > > >> > > > > > > Software Engineer Intern -
> > Confluent
> > > > >> > > > > > > > > > >> > > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > > > --
> > > > >> > > > > > > > > > >> > > > > > -- Guozhang
> > > > >> > > > > > > > > > >> > > > > >
> > > > >> > > > > > > > > > >> > > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > > > --
> > > > >> > > > > > > > > > >> > > > -- Guozhang
> > > > >> > > > > > > > > > >> > > >
> > > > >> > > > > > > > > > >> > >
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >> > --
> > > > >> > > > > > > > > > >> > -- Guozhang
> > > > >> > > > > > > > > > >> >
> > > > >> > > > > > > > > > >>
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > --
> > > > >> > > > > > > > > -- Guozhang
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > --
> > > > >> > > > > > > -- Guozhang
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jun Rao <ju...@confluent.io>.
Hi, Jason,

Thanks for the response. Should we make clusterId a nullable field
consistently in all new requests?

Jun

On Wed, Jul 29, 2020 at 12:20 PM Jason Gustafson <ja...@confluent.io> wrote:

> Hey Jun,
>
> I added a section on "Cluster Bootstrapping" which discusses clusterId
> generation and the process through which brokers find the current leader.
> The quick summary is that the first controller will be responsible for
> generating the clusterId and persisting it in the metadata log. Before the
> first leader has been elected, quorum APIs will skip clusterId validation.
> This seems reasonable since this is primarily intended to prevent the
> damage from misconfiguration after a cluster has been running for some
> time. Upon startup, brokers begin by sending Fetch requests to find the
> current leader. This will include the cluster.id from meta.properties if
> it
> is present. The broker will shutdown immediately if it receives
> INVALID_CLUSTER_ID from the Fetch response.
>
> I also added some details about our testing strategy, which you asked about
> previously.
>
> Thanks,
> Jason
>
> On Mon, Jul 27, 2020 at 10:46 PM Boyang Chen <re...@gmail.com>
> wrote:
>
> > On Mon, Jul 27, 2020 at 4:58 AM Unmesh Joshi <un...@gmail.com>
> > wrote:
> >
> > > Just checked etcd and zookeeper code, and both support leader to step
> > down
> > > as a follower to make sure there are no two leaders if the leader has
> > been
> > > disconnected from the majority of the followers
> > > For etcd this is https://github.com/etcd-io/etcd/issues/3866
> > > For Zookeeper its https://issues.apache.org/jira/browse/ZOOKEEPER-1699
> > > I was just thinking if it would be difficult to implement in the Pull
> > based
> > > model, but I guess not. It is possibly the same way ISR list is managed
> > > currently, if leader of the controller quorum loses majority of the
> > > followers, it should step down and become follower, that way, telling
> > > client in time that it was disconnected from the quorum, and not keep
> on
> > > sending state metadata to clients.
> > >
> > > Thanks,
> > > Unmesh
> > >
> > >
> > > On Mon, Jul 27, 2020 at 9:31 AM Unmesh Joshi <un...@gmail.com>
> > > wrote:
> > >
> > > > >>Could you clarify on this question? Which part of the raft group
> > > doesn't
> > > > >>know about leader dis-connection?
> > > > The leader of the controller quorum is partitioned from the
> controller
> > > > cluster, and a different leader is elected for the remaining
> controller
> > > > cluster.
> > >
> > I see your concern. For KIP-595 implementation, since there is no regular
> > heartbeats sent
> > from the leader to the followers, we decided to piggy-back on the fetch
> > timeout so that if the leader did not receive Fetch
> > requests from a majority of the quorum for that amount of time, it would
> > begin a new election and
> > start sending VoteRequest to voter nodes in the cluster to understand the
> > latest quorum. You could
> > find more details in this section
> > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP595:ARaftProtocolfortheMetadataQuorum-Vote
> > >
> > .
> >
> >
> > > > I think there are two things here,
> > > > 1.  The old leader will not know if it's disconnected from the rest
> of
> > > the
> > > > controller quorum cluster unless it receives BeginQuorumEpoch from
> the
> > > new
> > > > leader. So it will keep on serving stale metadata to the clients
> > > (Brokers,
> > > > Producers and Consumers)
> > > > 2. I assume, the Broker Leases will be managed on the controller
> quorum
> > > > leader. This partitioned leader will keep on tracking broker leases
> it
> > > has,
> > > > while the new leader of the quorum will also start managing broker
> > > leases.
> > > > So while the quorum leader is partitioned, there will be two
> membership
> > > > views of the kafka brokers managed on two leaders.
> > > > Unless broker heartbeats are also replicated as part of the Raft log,
> > > > there is no way to solve this?
> > > > I know LogCabin implementation does replicate client heartbeats. I
> > > suspect
> > > > that the same issue is there in Zookeeper, which does not replicate
> > > client
> > > > Ping requests..
> > > >
> > > > Thanks,
> > > > Unmesh
> > > >
> > > >
> > > >
> > > > On Mon, Jul 27, 2020 at 6:23 AM Boyang Chen <
> > reluctanthero104@gmail.com>
> > > > wrote:
> > > >
> > > >> Thanks for the questions Unmesh!
> > > >>
> > > >> On Sun, Jul 26, 2020 at 6:18 AM Unmesh Joshi <unmeshjoshi@gmail.com
> >
> > > >> wrote:
> > > >>
> > > >> > Hi,
> > > >> >
> > > >> > In the FetchRequest Handling, how to make sure we handle scenarios
> > > where
> > > >> > the leader might have been disconnected from the cluster, but
> > doesn't
> > > >> know
> > > >> > yet?
> > > >> >
> > > >> Could you clarify on this question? Which part of the raft group
> > doesn't
> > > >> know about leader
> > > >> dis-connection?
> > > >>
> > > >>
> > > >> > As discussed in the Raft Thesis section 6.4, the linearizable
> > > semantics
> > > >> of
> > > >> > read requests is implemented in LogCabin by sending heartbeat to
> > > >> followers
> > > >> > and waiting till the heartbeats are successful to make sure that
> the
> > > >> leader
> > > >> > is still the leader.
> > > >> > I think for the controller quorum to make sure none of the
> consumers
> > > get
> > > >> > stale data, it's important to have linearizable semantics? In the
> > pull
> > > >> > based model, the leader will need to wait for heartbeats from the
> > > >> followers
> > > >> > before returning each fetch request from the consumer then? Or do
> we
> > > >> need
> > > >> > to introduce some other request?
> > > >> > (Zookeeper does not have linearizable semantics for read requests,
> > but
> > > >> as
> > > >> > of now all the kafka interactions are through writes and watches).
> > > >> >
> > > >> > This is a very good question. For our v1 implementation we are not
> > > >> aiming
> > > >> to guarantee linearizable read, which
> > > >> would be considered as a follow-up effort. Note that today in Kafka
> > > there
> > > >> is no guarantee on the metadata freshness either,
> > > >> so no regression is introduced.
> > > >>
> > > >>
> > > >> > Thanks,
> > > >> > Unmesh
> > > >> >
> > > >> > On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io>
> wrote:
> > > >> >
> > > >> > > Hi, Jason,
> > > >> > >
> > > >> > > Thanks for the reply.
> > > >> > >
> > > >> > > 101. Sounds good. Regarding clusterId, I am not sure storing it
> in
> > > the
> > > >> > > metadata log is enough. For example, the vote request includes
> > > >> clusterId.
> > > >> > > So, no one can vote until they know the clusterId. Also, it
> would
> > be
> > > >> > useful
> > > >> > > to support the case when a voter completely loses its disk and
> > needs
> > > >> to
> > > >> > > recover.
> > > >> > >
> > > >> > > 210. There is no longer a FindQuorum request. When a follower
> > > >> restarts,
> > > >> > how
> > > >> > > does it discover the leader? Is that based on DescribeQuorum? It
> > > >> would be
> > > >> > > useful to document this.
> > > >> > >
> > > >> > > Jun
> > > >> > >
> > > >> > > On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <
> > jason@confluent.io
> > > >
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Hi Jun,
> > > >> > > >
> > > >> > > > Thanks for the questions.
> > > >> > > >
> > > >> > > > 101. I am treating some of the bootstrapping problems as out
> of
> > > the
> > > >> > scope
> > > >> > > > of this KIP. I am working on a separate proposal which
> addresses
> > > >> > > > bootstrapping security credentials specifically. Here is a
> rough
> > > >> sketch
> > > >> > > of
> > > >> > > > how I am seeing it:
> > > >> > > >
> > > >> > > > 1. Dynamic broker configurations including encrypted passwords
> > > will
> > > >> be
> > > >> > > > persisted in the metadata log and cached in the broker's
> > > >> > > `meta.properties`
> > > >> > > > file.
> > > >> > > > 2. We will provide a tool which allows users to directly
> > override
> > > >> the
> > > >> > > > values in `meta.properties` without requiring access to the
> > > quorum.
> > > >> > This
> > > >> > > > can be used to bootstrap the credentials of the voter set
> itself
> > > >> before
> > > >> > > the
> > > >> > > > cluster has been started.
> > > >> > > > 3. Some dynamic config changes will only be allowed when a
> > broker
> > > is
> > > >> > > > online. For example, changing a truststore password
> dynamically
> > > >> would
> > > >> > > > prevent that broker from being able to start if it were
> offline
> > > when
> > > >> > the
> > > >> > > > change was made.
> > > >> > > > 4. I am still thinking a little bit about SCRAM credentials,
> but
> > > >> most
> > > >> > > > likely they will be handled with an approach similar to
> > > >> > > `meta.properties`.
> > > >> > > >
> > > >> > > > 101.3 As for the question about `clusterId`, I think the way
> we
> > > >> would
> > > >> > do
> > > >> > > > this is to have the first elected leader generate a UUID and
> > write
> > > >> it
> > > >> > to
> > > >> > > > the metadata log. Let me add some detail to the proposal about
> > > this.
> > > >> > > >
> > > >> > > > A few additional answers below:
> > > >> > > >
> > > >> > > > 203. Yes, that is correct.
> > > >> > > >
> > > >> > > > 204. That is a good question. What happens in this case is
> that
> > > all
> > > >> > > voters
> > > >> > > > advance their epoch to the one designated by the candidate
> even
> > if
> > > >> they
> > > >> > > > reject its vote request. Assuming the candidate fails to be
> > > elected,
> > > >> > the
> > > >> > > > election will be retried until a leader emerges.
> > > >> > > >
> > > >> > > > 205. I had some discussion with Colin offline about this
> > problem.
> > > I
> > > >> > think
> > > >> > > > the answer should be "yes," but it probably needs a little
> more
> > > >> > thought.
> > > >> > > > Handling JBOD failures is tricky. For an observer, we can
> > > replicate
> > > >> the
> > > >> > > > metadata log from scratch safely in a new log dir. But if the
> > log
> > > >> dir
> > > >> > of
> > > >> > > a
> > > >> > > > voter fails, I do not think it is generally safe to start from
> > an
> > > >> empty
> > > >> > > > state.
> > > >> > > >
> > > >> > > > 206. Yes, that is discussed in KIP-631 I believe.
> > > >> > > >
> > > >> > > > 207. Good suggestion. I will work on this.
> > > >> > > >
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > > Jason
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io>
> > wrote:
> > > >> > > >
> > > >> > > > > Hi, Jason,
> > > >> > > > >
> > > >> > > > > Thanks for the updated KIP. Looks good overall. A few more
> > > >> comments
> > > >> > > > below.
> > > >> > > > >
> > > >> > > > > 101. I still don't see a section on bootstrapping related
> > > issues.
> > > >> It
> > > >> > > > would
> > > >> > > > > be useful to document if/how the following is supported.
> > > >> > > > > 101.1 Currently, we support auto broker id generation. Is
> this
> > > >> > > supported
> > > >> > > > > for bootstrap brokers?
> > > >> > > > > 101.2 As Colin mentioned, sometimes we may need to load the
> > > >> security
> > > >> > > > > credentials to be broker before it can be connected to.
> Could
> > > you
> > > >> > > > provide a
> > > >> > > > > bit more detail on how this will work?
> > > >> > > > > 101.3 Currently, we use ZK to generate clusterId on a new
> > > cluster.
> > > >> > With
> > > >> > > > > Raft, how does every broker generate the same clusterId in a
> > > >> > > distributed
> > > >> > > > > way?
> > > >> > > > >
> > > >> > > > > 200. It would be useful to document if the various special
> > > offsets
> > > >> > (log
> > > >> > > > > start offset, recovery point, HWM, etc) for the Raft log are
> > > >> stored
> > > >> > in
> > > >> > > > the
> > > >> > > > > same existing checkpoint files or not.
> > > >> > > > > 200.1 Since the Raft log flushes every append, does that
> allow
> > > us
> > > >> to
> > > >> > > > > recover from a recovery point within the active segment or
> do
> > we
> > > >> > still
> > > >> > > > need
> > > >> > > > > to scan the full segment including the recovery point? The
> > > former
> > > >> can
> > > >> > > be
> > > >> > > > > tricky since multiple records can fall into the same disk
> page
> > > >> and a
> > > >> > > > > subsequent flush may corrupt a page with previously flushed
> > > >> records.
> > > >> > > > >
> > > >> > > > > 201. Configurations.
> > > >> > > > > 201.1 How do the Raft brokers get security related configs
> for
> > > >> inter
> > > >> > > > broker
> > > >> > > > > communication? Is that based on the existing
> > > >> > > > > inter.broker.security.protocol?
> > > >> > > > > 201.2 We have quorum.retry.backoff.max.ms and
> > > >> > quorum.retry.backoff.ms,
> > > >> > > > but
> > > >> > > > > only quorum.election.backoff.max.ms. This seems a bit
> > > >> inconsistent.
> > > >> > > > >
> > > >> > > > > 202. Metrics:
> > > >> > > > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
> > > >> > > OutboundQueueTimeMs:
> > > >> > > > > Are those the same as existing totalTime, requestQueueTime,
> > > >> > localTime,
> > > >> > > > > responseQueueTime? Could we reuse the existing ones with the
> > tag
> > > >> > > > > request=[request-type]?
> > > >> > > > > 202.2. Could you explain what InboundChannelSize and
> > > >> > > OutboundChannelSize
> > > >> > > > > are?
> > > >> > > > > 202.3 ElectionLatencyMax/Avg: It seems that both should be
> > > >> windowed?
> > > >> > > > >
> > > >> > > > > 203. Quorum State: I assume that LeaderId will be kept
> > > >> consistently
> > > >> > > with
> > > >> > > > > LeaderEpoch. For example, if a follower transitions to
> > candidate
> > > >> and
> > > >> > > > bumps
> > > >> > > > > up LeaderEpoch, it will set leaderId to -1 and persist both
> in
> > > the
> > > >> > > Quorum
> > > >> > > > > state file. Is that correct?
> > > >> > > > >
> > > >> > > > > 204. I was thinking about a corner case when a Raft broker
> is
> > > >> > > partitioned
> > > >> > > > > off. This broker will then be in a continuous loop of
> bumping
> > up
> > > >> the
> > > >> > > > leader
> > > >> > > > > epoch, but failing to get enough votes. When the
> partitioning
> > is
> > > >> > > removed,
> > > >> > > > > this broker's high leader epoch will force a leader
> election.
> > I
> > > >> > assume
> > > >> > > > > other Raft brokers can immediately advance their leader
> epoch
> > > >> passing
> > > >> > > the
> > > >> > > > > already bumped epoch such that leader election won't be
> > delayed.
> > > >> Is
> > > >> > > that
> > > >> > > > > right?
> > > >> > > > >
> > > >> > > > > 205. In a JBOD setting, could we use the existing tool to
> move
> > > the
> > > >> > Raft
> > > >> > > > log
> > > >> > > > > from one disk to another?
> > > >> > > > >
> > > >> > > > > 206. The KIP doesn't mention the local metadata store
> derived
> > > from
> > > >> > the
> > > >> > > > Raft
> > > >> > > > > log. Will that be covered in a separate KIP?
> > > >> > > > >
> > > >> > > > > 207. Since this is a critical component. Could we add a
> > section
> > > on
> > > >> > the
> > > >> > > > > testing plan for correctness?
> > > >> > > > >
> > > >> > > > > 208. Performance. Do we plan to do group commit (e.g. buffer
> > > >> pending
> > > >> > > > > appends during a flush and then flush all accumulated
> pending
> > > >> records
> > > >> > > > > together in the next flush) for better throughput?
> > > >> > > > >
> > > >> > > > > 209. "the leader can actually defer fsync until it knows
> > > >> > "quorum.size -
> > > >> > > > 1"
> > > >> > > > > has get to a certain entry offset." Why is that
> "quorum.size -
> > > 1"
> > > >> > > instead
> > > >> > > > > of the majority of the quorum?
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > >
> > > >> > > > > Jun
> > > >> > > > >
> > > >> > > > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <
> > > >> jason@confluent.io>
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > > > Hi All,
> > > >> > > > > >
> > > >> > > > > > Just a quick update on the proposal. We have decided to
> move
> > > >> quorum
> > > >> > > > > > reassignment to a separate KIP:
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > > >> > > > > > .
> > > >> > > > > > The way this ties into cluster bootstrapping is
> complicated,
> > > so
> > > >> we
> > > >> > > felt
> > > >> > > > > we
> > > >> > > > > > needed a bit more time for validation. That leaves the
> core
> > of
> > > >> this
> > > >> > > > > > proposal as quorum-based replication. If there are no
> > further
> > > >> > > comments,
> > > >> > > > > we
> > > >> > > > > > will plan to start a vote later this week.
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Jason
> > > >> > > > > >
> > > >> > > > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <
> > > >> wangguoz@gmail.com
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > > >
> > > >> > > > > > > @Jun Rao <ju...@gmail.com>
> > > >> > > > > > >
> > > >> > > > > > > Regarding your comment about log compaction. After some
> > > >> > deep-diving
> > > >> > > > > into
> > > >> > > > > > > this we've decided to propose a new snapshot-based log
> > > >> cleaning
> > > >> > > > > mechanism
> > > >> > > > > > > which would be used to replace the current compaction
> > > >> mechanism
> > > >> > for
> > > >> > > > > this
> > > >> > > > > > > meta log. A new KIP will be proposed specifically for
> this
> > > >> idea.
> > > >> > > > > > >
> > > >> > > > > > > All,
> > > >> > > > > > >
> > > >> > > > > > > I've updated the KIP wiki a bit updating one config "
> > > >> > > > > > > election.jitter.max.ms"
> > > >> > > > > > > to "election.backoff.max.ms" to make it more clear
> about
> > > the
> > > >> > > usage:
> > > >> > > > > the
> > > >> > > > > > > configured value will be the upper bound of the binary
> > > >> > exponential
> > > >> > > > > > backoff
> > > >> > > > > > > time after a failed election, before starting a new one.
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > Guozhang
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> > > >> > > > > reluctanthero104@gmail.com>
> > > >> > > > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Thanks for the suggestions Guozhang.
> > > >> > > > > > > >
> > > >> > > > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
> > > >> > > wangguoz@gmail.com>
> > > >> > > > > > > wrote:
> > > >> > > > > > > >
> > > >> > > > > > > > > Hello Boyang,
> > > >> > > > > > > > >
> > > >> > > > > > > > > Thanks for the updated information. A few questions
> > > here:
> > > >> > > > > > > > >
> > > >> > > > > > > > > 1) Should the quorum-file also update to support
> > > >> multi-raft?
> > > >> > > > > > > > >
> > > >> > > > > > > > > I'm neutral about this, as we don't know yet how the
> > > >> > multi-raft
> > > >> > > > > > modules
> > > >> > > > > > > > would behave. If
> > > >> > > > > > > > we have different threads operating different raft
> > groups,
> > > >> > > > > > consolidating
> > > >> > > > > > > > the `checkpoint` files seems
> > > >> > > > > > > > not reasonable. We could always add
> `multi-quorum-file`
> > > >> later
> > > >> > if
> > > >> > > > > > > possible.
> > > >> > > > > > > >
> > > >> > > > > > > > 2) In the previous proposal, there's fields in the
> > > >> > > > FetchQuorumRecords
> > > >> > > > > > > like
> > > >> > > > > > > > > latestDirtyOffset, is that dropped intentionally?
> > > >> > > > > > > > >
> > > >> > > > > > > > > I dropped the latestDirtyOffset since it is
> associated
> > > >> with
> > > >> > the
> > > >> > > > log
> > > >> > > > > > > > compaction discussion. This is beyond this KIP scope
> and
> > > we
> > > >> > could
> > > >> > > > > > > > potentially get a separate KIP to talk about it.
> > > >> > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > > > > 3) I think we also need to elaborate a bit more
> > details
> > > >> > > regarding
> > > >> > > > > > when
> > > >> > > > > > > to
> > > >> > > > > > > > > send metadata request and discover-brokers;
> currently
> > we
> > > >> only
> > > >> > > > > > discussed
> > > >> > > > > > > > > during bootstrap how these requests would be sent. I
> > > think
> > > >> > the
> > > >> > > > > > > following
> > > >> > > > > > > > > scenarios would also need these requests
> > > >> > > > > > > > >
> > > >> > > > > > > > > 3.a) As long as a broker does not know the current
> > > quorum
> > > >> > > > > (including
> > > >> > > > > > > the
> > > >> > > > > > > > > leader and the voters), it should continue
> > periodically
> > > >> ask
> > > >> > > other
> > > >> > > > > > > brokers
> > > >> > > > > > > > > via "metadata.
> > > >> > > > > > > > > 3.b) As long as a broker does not know all the
> current
> > > >> quorum
> > > >> > > > > voter's
> > > >> > > > > > > > > connections, it should continue periodically ask
> other
> > > >> > brokers
> > > >> > > > via
> > > >> > > > > > > > > "discover-brokers".
> > > >> > > > > > > > > 3.c) When the leader's fetch timeout elapsed, it
> > should
> > > >> send
> > > >> > > > > metadata
> > > >> > > > > > > > > request.
> > > >> > > > > > > > >
> > > >> > > > > > > > > Make sense, will add to the KIP.
> > > >> > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > > > Guozhang
> > > >> > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > >> > > > > > > reluctanthero104@gmail.com>
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > >
> > > >> > > > > > > > > > Hey all,
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > follow-up on the previous email, we made some more
> > > >> updates:
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > 1. The Alter/DescribeQuorum RPCs are also
> > > re-structured
> > > >> to
> > > >> > > use
> > > >> > > > > > > > > multi-raft.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > 2. We add observer status into the
> > > >> DescribeQuorumResponse
> > > >> > as
> > > >> > > we
> > > >> > > > > see
> > > >> > > > > > > it
> > > >> > > > > > > > > is a
> > > >> > > > > > > > > > low hanging fruit which is very useful for user
> > > >> debugging
> > > >> > and
> > > >> > > > > > > > > reassignment.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > 3. The FindQuorum RPC is replaced with
> > DiscoverBrokers
> > > >> RPC,
> > > >> > > > which
> > > >> > > > > > is
> > > >> > > > > > > > > purely
> > > >> > > > > > > > > > in charge of discovering broker connections in a
> > > gossip
> > > >> > > manner.
> > > >> > > > > The
> > > >> > > > > > > > > quorum
> > > >> > > > > > > > > > leader discovery is piggy-back on the Metadata RPC
> > for
> > > >> the
> > > >> > > > topic
> > > >> > > > > > > > > partition
> > > >> > > > > > > > > > leader, which in our case is the single metadata
> > > >> partition
> > > >> > > for
> > > >> > > > > the
> > > >> > > > > > > > > version
> > > >> > > > > > > > > > one.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > Let me know if you have any questions.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > Boyang
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > >> > > > > > > > reluctanthero104@gmail.com>
> > > >> > > > > > > > > > wrote:
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > > Hey all,
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > Thanks for the great discussions so far. I'm
> > posting
> > > >> some
> > > >> > > KIP
> > > >> > > > > > > updates
> > > >> > > > > > > > > > from
> > > >> > > > > > > > > > > our working group discussion:
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > 1. We will be changing the core RPCs from
> > > single-raft
> > > >> API
> > > >> > > to
> > > >> > > > > > > > > multi-raft.
> > > >> > > > > > > > > > > This means all protocols will be "batch" in the
> > > first
> > > >> > > > version,
> > > >> > > > > > but
> > > >> > > > > > > > the
> > > >> > > > > > > > > > KIP
> > > >> > > > > > > > > > > itself only illustrates the design for a single
> > > >> metadata
> > > >> > > > topic
> > > >> > > > > > > > > partition.
> > > >> > > > > > > > > > > The reason is to "keep the door open" for future
> > > >> > extensions
> > > >> > > > of
> > > >> > > > > > this
> > > >> > > > > > > > > piece
> > > >> > > > > > > > > > > of module such as a sharded controller or
> general
> > > >> quorum
> > > >> > > > based
> > > >> > > > > > > topic
> > > >> > > > > > > > > > > replication, beyond the current Kafka
> replication
> > > >> > protocol.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > 2. We will piggy-back on the current Kafka Fetch
> > API
> > > >> > > instead
> > > >> > > > of
> > > >> > > > > > > > > inventing
> > > >> > > > > > > > > > > a new FetchQuorumRecords RPC. The motivation is
> > > about
> > > >> the
> > > >> > > > same
> > > >> > > > > as
> > > >> > > > > > > #1
> > > >> > > > > > > > as
> > > >> > > > > > > > > > > well as making the integration work easier,
> > instead
> > > of
> > > >> > > > letting
> > > >> > > > > > two
> > > >> > > > > > > > > > similar
> > > >> > > > > > > > > > > RPCs diverge.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of
> only
> > > >> > sending
> > > >> > > > the
> > > >> > > > > > > > request
> > > >> > > > > > > > > to
> > > >> > > > > > > > > > > the most caught-up voter, we shall broadcast the
> > > >> > > information
> > > >> > > > to
> > > >> > > > > > all
> > > >> > > > > > > > > > voters,
> > > >> > > > > > > > > > > with a sorted voter list in descending order of
> > > their
> > > >> > > > > > corresponding
> > > >> > > > > > > > > > > replicated offset. In this way, the top voter
> will
> > > >> > become a
> > > >> > > > > > > candidate
> > > >> > > > > > > > > > > immediately, while the other voters shall wait
> for
> > > an
> > > >> > > > > exponential
> > > >> > > > > > > > > > back-off
> > > >> > > > > > > > > > > to trigger elections, which helps ensure the top
> > > voter
> > > >> > gets
> > > >> > > > > > > elected,
> > > >> > > > > > > > > and
> > > >> > > > > > > > > > > the election eventually happens when the top
> voter
> > > is
> > > >> not
> > > >> > > > > > > responsive.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > Please see the updated KIP and post any
> questions
> > or
> > > >> > > concerns
> > > >> > > > > on
> > > >> > > > > > > the
> > > >> > > > > > > > > > > mailing thread.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > Boyang
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <
> > > >> jun@confluent.io
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > >> Hi, Guozhang and Jason,
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > > >> Thanks for the reply. A couple of more replies.
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > > >> 102. Still not sure about this. How is the
> > > tombstone
> > > >> > issue
> > > >> > > > > > > addressed
> > > >> > > > > > > > > in
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> non-voter and the observer.  They can die at
> any
> > > >> point
> > > >> > and
> > > >> > > > > > restart
> > > >> > > > > > > > at
> > > >> > > > > > > > > an
> > > >> > > > > > > > > > >> arbitrary later time, and the advancing of the
> > > >> > firstDirty
> > > >> > > > > offset
> > > >> > > > > > > and
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > >> removal of the tombstone can happen
> > independently.
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > > >> 106. I agree that it would be less confusing if
> > we
> > > >> used
> > > >> > > > > "epoch"
> > > >> > > > > > > > > instead
> > > >> > > > > > > > > > of
> > > >> > > > > > > > > > >> "leader epoch" consistently.
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > > >> Jun
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > > >> > > > > > wangguoz@gmail.com>
> > > >> > > > > > > > > > wrote:
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
> > > >> > > jun@confluent.io
> > > >> > > > >
> > > >> > > > > > > wrote:
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > > Hi, Guozhang,
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > Thanks for the reply. A few more replies
> > > inlined
> > > >> > > below.
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang
> Wang
> > <
> > > >> > > > > > > > wangguoz@gmail.com>
> > > >> > > > > > > > > > >> wrote:
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > > Hello Jun,
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > Thanks for your comments! I'm replying
> > inline
> > > >> > below:
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> > > >> > > > > jun@confluent.io
> > > >> > > > > > >
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > > 101. Bootstrapping related issues.
> > > >> > > > > > > > > > >> > > > > 101.1 Currently, we support auto broker
> > id
> > > >> > > > generation.
> > > >> > > > > > Is
> > > >> > > > > > > > this
> > > >> > > > > > > > > > >> > > supported
> > > >> > > > > > > > > > >> > > > > for bootstrap brokers?
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > The vote ids would just be the broker
> ids.
> > > >> > > > > > > "bootstrap.servers"
> > > >> > > > > > > > > > >> would be
> > > >> > > > > > > > > > >> > > > similar to what client configs have
> today,
> > > >> where
> > > >> > > > > > > > "quorum.voters"
> > > >> > > > > > > > > > >> would
> > > >> > > > > > > > > > >> > be
> > > >> > > > > > > > > > >> > > > pre-defined config values.
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > My question was on the auto generated
> broker
> > > id.
> > > >> > > > > Currently,
> > > >> > > > > > > the
> > > >> > > > > > > > > > broker
> > > >> > > > > > > > > > >> > can
> > > >> > > > > > > > > > >> > > choose to have its broker Id auto
> generated.
> > > The
> > > >> > > > > generation
> > > >> > > > > > is
> > > >> > > > > > > > > done
> > > >> > > > > > > > > > >> > through
> > > >> > > > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK,
> it's
> > > not
> > > >> > clear
> > > >> > > > how
> > > >> > > > > > the
> > > >> > > > > > > > > > broker
> > > >> > > > > > > > > > >> id
> > > >> > > > > > > > > > >> > is
> > > >> > > > > > > > > > >> > > auto generated. "quorum.voters" also can't
> be
> > > set
> > > >> > > > > statically
> > > >> > > > > > > if
> > > >> > > > > > > > > > broker
> > > >> > > > > > > > > > >> > ids
> > > >> > > > > > > > > > >> > > are auto generated.
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > Jason has explained some ideas that we've
> > > >> discussed
> > > >> > so
> > > >> > > > > far,
> > > >> > > > > > > the
> > > >> > > > > > > > > > >> reason we
> > > >> > > > > > > > > > >> > intentional did not include them so far is
> that
> > > we
> > > >> > feel
> > > >> > > it
> > > >> > > > > is
> > > >> > > > > > > > > out-side
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > scope of KIP-595. Under the umbrella of
> KIP-500
> > > we
> > > >> > > should
> > > >> > > > > > > > definitely
> > > >> > > > > > > > > > >> > address them though.
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > On the high-level, our belief is that
> "joining
> > a
> > > >> > quorum"
> > > >> > > > and
> > > >> > > > > > > > > "joining
> > > >> > > > > > > > > > >> (or
> > > >> > > > > > > > > > >> > more specifically, registering brokers in)
> the
> > > >> > cluster"
> > > >> > > > > would
> > > >> > > > > > be
> > > >> > > > > > > > > > >> > de-coupled a bit, where the former should be
> > > >> completed
> > > >> > > > > before
> > > >> > > > > > we
> > > >> > > > > > > > do
> > > >> > > > > > > > > > the
> > > >> > > > > > > > > > >> > latter. More specifically, assuming the
> quorum
> > is
> > > >> > > already
> > > >> > > > up
> > > >> > > > > > and
> > > >> > > > > > > > > > >> running,
> > > >> > > > > > > > > > >> > after the newly started broker found the
> leader
> > > of
> > > >> the
> > > >> > > > > quorum
> > > >> > > > > > it
> > > >> > > > > > > > can
> > > >> > > > > > > > > > >> send a
> > > >> > > > > > > > > > >> > specific RegisterBroker request including its
> > > >> > listener /
> > > >> > > > > > > protocol
> > > >> > > > > > > > /
> > > >> > > > > > > > > > etc,
> > > >> > > > > > > > > > >> > and upon handling it the leader can send back
> > the
> > > >> > > uniquely
> > > >> > > > > > > > generated
> > > >> > > > > > > > > > >> broker
> > > >> > > > > > > > > > >> > id to the new broker, while also executing
> the
> > > >> > > > > > "startNewBroker"
> > > >> > > > > > > > > > >> callback as
> > > >> > > > > > > > > > >> > the controller.
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > > > 102. Log compaction. One weak spot of
> log
> > > >> > > compaction
> > > >> > > > > is
> > > >> > > > > > > for
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > >> > > consumer
> > > >> > > > > > > > > > >> > > > to
> > > >> > > > > > > > > > >> > > > > deal with deletes. When a key is
> deleted,
> > > >> it's
> > > >> > > > > retained
> > > >> > > > > > > as a
> > > >> > > > > > > > > > >> > tombstone
> > > >> > > > > > > > > > >> > > > > first and then physically removed. If a
> > > >> client
> > > >> > > > misses
> > > >> > > > > > the
> > > >> > > > > > > > > > >> tombstone
> > > >> > > > > > > > > > >> > > > > (because it's physically removed), it
> may
> > > >> not be
> > > >> > > > able
> > > >> > > > > to
> > > >> > > > > > > > > update
> > > >> > > > > > > > > > >> its
> > > >> > > > > > > > > > >> > > > > metadata properly. The way we solve
> this
> > in
> > > >> > Kafka
> > > >> > > is
> > > >> > > > > > based
> > > >> > > > > > > > on
> > > >> > > > > > > > > a
> > > >> > > > > > > > > > >> > > > > configuration (
> > > >> log.cleaner.delete.retention.ms)
> > > >> > > and
> > > >> > > > > we
> > > >> > > > > > > > > expect a
> > > >> > > > > > > > > > >> > > consumer
> > > >> > > > > > > > > > >> > > > > having seen an old key to finish
> reading
> > > the
> > > >> > > > deletion
> > > >> > > > > > > > > tombstone
> > > >> > > > > > > > > > >> > within
> > > >> > > > > > > > > > >> > > > that
> > > >> > > > > > > > > > >> > > > > time. There is no strong guarantee for
> > that
> > > >> > since
> > > >> > > a
> > > >> > > > > > broker
> > > >> > > > > > > > > could
> > > >> > > > > > > > > > >> be
> > > >> > > > > > > > > > >> > > down
> > > >> > > > > > > > > > >> > > > > for a long time. It would be better if
> we
> > > can
> > > >> > > have a
> > > >> > > > > > more
> > > >> > > > > > > > > > reliable
> > > >> > > > > > > > > > >> > way
> > > >> > > > > > > > > > >> > > of
> > > >> > > > > > > > > > >> > > > > dealing with deletes.
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > We propose to capture this in the
> > > >> > "FirstDirtyOffset"
> > > >> > > > > field
> > > >> > > > > > > of
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > >> > quorum
> > > >> > > > > > > > > > >> > > > record fetch response: the offset is the
> > > >> maximum
> > > >> > > > offset
> > > >> > > > > > that
> > > >> > > > > > > > log
> > > >> > > > > > > > > > >> > > compaction
> > > >> > > > > > > > > > >> > > > has reached up to. If the follower has
> > > fetched
> > > >> > > beyond
> > > >> > > > > this
> > > >> > > > > > > > > offset
> > > >> > > > > > > > > > it
> > > >> > > > > > > > > > >> > > means
> > > >> > > > > > > > > > >> > > > itself is safe hence it has seen all
> > records
> > > >> up to
> > > >> > > > that
> > > >> > > > > > > > offset.
> > > >> > > > > > > > > On
> > > >> > > > > > > > > > >> > > getting
> > > >> > > > > > > > > > >> > > > the response, the follower can then
> decide
> > if
> > > >> its
> > > >> > > end
> > > >> > > > > > offset
> > > >> > > > > > > > > > >> actually
> > > >> > > > > > > > > > >> > > below
> > > >> > > > > > > > > > >> > > > that dirty offset (and hence may miss
> some
> > > >> > > > tombstones).
> > > >> > > > > If
> > > >> > > > > > > > > that's
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > > case:
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > 1) Naively, it could re-bootstrap
> metadata
> > > log
> > > >> > from
> > > >> > > > the
> > > >> > > > > > very
> > > >> > > > > > > > > > >> beginning
> > > >> > > > > > > > > > >> > to
> > > >> > > > > > > > > > >> > > > catch up.
> > > >> > > > > > > > > > >> > > > 2) During that time, it would refrain
> > itself
> > > >> from
> > > >> > > > > > answering
> > > >> > > > > > > > > > >> > > MetadataRequest
> > > >> > > > > > > > > > >> > > > from any clients.
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > I am not sure that the "FirstDirtyOffset"
> > field
> > > >> > fully
> > > >> > > > > > > addresses
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > >> > issue.
> > > >> > > > > > > > > > >> > > Currently, the deletion tombstone is not
> > > removed
> > > >> > > > > immediately
> > > >> > > > > > > > > after a
> > > >> > > > > > > > > > >> > round
> > > >> > > > > > > > > > >> > > of cleaning. It's removed after a delay in
> a
> > > >> > > subsequent
> > > >> > > > > > round
> > > >> > > > > > > of
> > > >> > > > > > > > > > >> > cleaning.
> > > >> > > > > > > > > > >> > > Consider an example where a key insertion
> is
> > at
> > > >> > offset
> > > >> > > > 200
> > > >> > > > > > > and a
> > > >> > > > > > > > > > >> deletion
> > > >> > > > > > > > > > >> > > tombstone of the key is at 400. Initially,
> > > >> > > > > FirstDirtyOffset
> > > >> > > > > > is
> > > >> > > > > > > > at
> > > >> > > > > > > > > > >> 300. A
> > > >> > > > > > > > > > >> > > follower/observer fetches from offset 0
> and
> > > >> fetches
> > > >> > > the
> > > >> > > > > key
> > > >> > > > > > > at
> > > >> > > > > > > > > > offset
> > > >> > > > > > > > > > >> > 200.
> > > >> > > > > > > > > > >> > > A few rounds of cleaning happen.
> > > >> FirstDirtyOffset is
> > > >> > > at
> > > >> > > > > 500
> > > >> > > > > > > and
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > >> > > tombstone at 400 is physically removed. The
> > > >> > > > > > follower/observer
> > > >> > > > > > > > > > >> continues
> > > >> > > > > > > > > > >> > the
> > > >> > > > > > > > > > >> > > fetch, but misses offset 400. It catches
> all
> > > the
> > > >> way
> > > >> > > to
> > > >> > > > > > > > > > >> FirstDirtyOffset
> > > >> > > > > > > > > > >> > > and declares its metadata as ready.
> However,
> > > its
> > > >> > > > metadata
> > > >> > > > > > > could
> > > >> > > > > > > > be
> > > >> > > > > > > > > > >> stale
> > > >> > > > > > > > > > >> > > since it actually misses the deletion of
> the
> > > key.
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > Yeah good question, I should have put more
> > > >> details
> > > >> > in
> > > >> > > my
> > > >> > > > > > > > > explanation
> > > >> > > > > > > > > > >> :)
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > The idea is that we will adjust the log
> > > compaction
> > > >> for
> > > >> > > > this
> > > >> > > > > > raft
> > > >> > > > > > > > > based
> > > >> > > > > > > > > > >> > metadata log: before more details to be
> > > explained,
> > > >> > since
> > > >> > > > we
> > > >> > > > > > have
> > > >> > > > > > > > two
> > > >> > > > > > > > > > >> types
> > > >> > > > > > > > > > >> > of "watermarks" here, whereas in Kafka the
> > > >> watermark
> > > >> > > > > indicates
> > > >> > > > > > > > where
> > > >> > > > > > > > > > >> every
> > > >> > > > > > > > > > >> > replica have replicated up to and in Raft the
> > > >> > watermark
> > > >> > > > > > > indicates
> > > >> > > > > > > > > > where
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > majority of replicas (here only indicating
> > voters
> > > >> of
> > > >> > the
> > > >> > > > > > quorum,
> > > >> > > > > > > > not
> > > >> > > > > > > > > > >> > counting observers) have replicated up to,
> > let's
> > > >> call
> > > >> > > them
> > > >> > > > > > Kafka
> > > >> > > > > > > > > > >> watermark
> > > >> > > > > > > > > > >> > and Raft watermark. For this special log, we
> > > would
> > > >> > > > maintain
> > > >> > > > > > both
> > > >> > > > > > > > > > >> > watermarks.
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > When log compacting on the leader, we would
> > only
> > > >> > compact
> > > >> > > > up
> > > >> > > > > to
> > > >> > > > > > > the
> > > >> > > > > > > > > > Kafka
> > > >> > > > > > > > > > >> > watermark, i.e. if there is at least one
> voter
> > > who
> > > >> > have
> > > >> > > > not
> > > >> > > > > > > > > replicated
> > > >> > > > > > > > > > >> an
> > > >> > > > > > > > > > >> > entry, it would not be compacted. The
> > > >> "dirty-offset"
> > > >> > is
> > > >> > > > the
> > > >> > > > > > > offset
> > > >> > > > > > > > > > that
> > > >> > > > > > > > > > >> > we've compacted up to and is communicated to
> > > other
> > > >> > > voters,
> > > >> > > > > and
> > > >> > > > > > > the
> > > >> > > > > > > > > > other
> > > >> > > > > > > > > > >> > voters would also compact up to this value
> ---
> > > i.e.
> > > >> > the
> > > >> > > > > > > difference
> > > >> > > > > > > > > > here
> > > >> > > > > > > > > > >> is
> > > >> > > > > > > > > > >> > that instead of letting each replica doing
> log
> > > >> > > compaction
> > > >> > > > > > > > > > independently,
> > > >> > > > > > > > > > >> > we'll have the leader to decide upon which
> > offset
> > > >> to
> > > >> > > > compact
> > > >> > > > > > to,
> > > >> > > > > > > > and
> > > >> > > > > > > > > > >> > propagate this value to others to follow, in
> a
> > > more
> > > >> > > > > > coordinated
> > > >> > > > > > > > > > manner.
> > > >> > > > > > > > > > >> > Also note when there are new voters joining
> the
> > > >> quorum
> > > >> > > who
> > > >> > > > > has
> > > >> > > > > > > not
> > > >> > > > > > > > > > >> > replicated up to the dirty-offset, of because
> > of
> > > >> other
> > > >> > > > > issues
> > > >> > > > > > > they
> > > >> > > > > > > > > > >> > truncated their logs to below the
> dirty-offset,
> > > >> they'd
> > > >> > > > have
> > > >> > > > > to
> > > >> > > > > > > > > > >> re-bootstrap
> > > >> > > > > > > > > > >> > from the beginning, and during this period of
> > > time
> > > >> the
> > > >> > > > > leader
> > > >> > > > > > > > > learned
> > > >> > > > > > > > > > >> about
> > > >> > > > > > > > > > >> > this lagging voter would not advance the
> > > watermark
> > > >> > (also
> > > >> > > > it
> > > >> > > > > > > would
> > > >> > > > > > > > > not
> > > >> > > > > > > > > > >> > decrement it), and hence not compacting
> either,
> > > >> until
> > > >> > > the
> > > >> > > > > > > voter(s)
> > > >> > > > > > > > > has
> > > >> > > > > > > > > > >> > caught up to that dirty-offset.
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > So back to your example above, before the
> > > bootstrap
> > > >> > > voter
> > > >> > > > > gets
> > > >> > > > > > > to
> > > >> > > > > > > > > 300
> > > >> > > > > > > > > > no
> > > >> > > > > > > > > > >> > log compaction would happen on the leader;
> and
> > > >> until
> > > >> > > later
> > > >> > > > > > when
> > > >> > > > > > > > the
> > > >> > > > > > > > > > >> voter
> > > >> > > > > > > > > > >> > have got to beyond 400 and hence replicated
> > that
> > > >> > > > tombstone,
> > > >> > > > > > the
> > > >> > > > > > > > log
> > > >> > > > > > > > > > >> > compaction would possibly get to that
> tombstone
> > > and
> > > >> > > remove
> > > >> > > > > it.
> > > >> > > > > > > Say
> > > >> > > > > > > > > > >> later it
> > > >> > > > > > > > > > >> > the leader's log compaction reaches 500, it
> can
> > > >> send
> > > >> > > this
> > > >> > > > > back
> > > >> > > > > > > to
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > >> voter
> > > >> > > > > > > > > > >> > who can then also compact locally up to 500.
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > > > > 105. Quorum State: In addition to
> > VotedId,
> > > >> do we
> > > >> > > > need
> > > >> > > > > > the
> > > >> > > > > > > > > epoch
> > > >> > > > > > > > > > >> > > > > corresponding to VotedId? Over time,
> the
> > > same
> > > >> > > broker
> > > >> > > > > Id
> > > >> > > > > > > > could
> > > >> > > > > > > > > be
> > > >> > > > > > > > > > >> > voted
> > > >> > > > > > > > > > >> > > in
> > > >> > > > > > > > > > >> > > > > different generations with different
> > epoch.
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > > Hmm, this is a good point. Originally I
> > think
> > > >> the
> > > >> > > > > > > > "LeaderEpoch"
> > > >> > > > > > > > > > >> field
> > > >> > > > > > > > > > >> > in
> > > >> > > > > > > > > > >> > > > that file is corresponding to the "latest
> > > known
> > > >> > > leader
> > > >> > > > > > > epoch",
> > > >> > > > > > > > > not
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > > > "current leader epoch". For example, if
> the
> > > >> > current
> > > >> > > > > epoch
> > > >> > > > > > is
> > > >> > > > > > > > N,
> > > >> > > > > > > > > > and
> > > >> > > > > > > > > > >> > then
> > > >> > > > > > > > > > >> > > a
> > > >> > > > > > > > > > >> > > > vote-request with epoch N+1 is received
> and
> > > the
> > > >> > > voter
> > > >> > > > > > > granted
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > >> vote
> > > >> > > > > > > > > > >> > > for
> > > >> > > > > > > > > > >> > > > it, then it means for this voter it knows
> > the
> > > >> > > "latest
> > > >> > > > > > epoch"
> > > >> > > > > > > > is
> > > >> > > > > > > > > N
> > > >> > > > > > > > > > +
> > > >> > > > > > > > > > >> 1
> > > >> > > > > > > > > > >> > > > although it is unknown if that sending
> > > >> candidate
> > > >> > > will
> > > >> > > > > > indeed
> > > >> > > > > > > > > > become
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > > new
> > > >> > > > > > > > > > >> > > > leader (which would only be notified via
> > > >> > > begin-quorum
> > > >> > > > > > > > request).
> > > >> > > > > > > > > > >> > However,
> > > >> > > > > > > > > > >> > > > when persisting the quorum state, we
> would
> > > >> encode
> > > >> > > > > > > leader-epoch
> > > >> > > > > > > > > to
> > > >> > > > > > > > > > >> N+1,
> > > >> > > > > > > > > > >> > > > while the leaderId to be the older
> leader.
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > But now thinking about this a bit more, I
> > > feel
> > > >> we
> > > >> > > > should
> > > >> > > > > > use
> > > >> > > > > > > > two
> > > >> > > > > > > > > > >> > separate
> > > >> > > > > > > > > > >> > > > epochs, one for the "lates known" and one
> > for
> > > >> the
> > > >> > > > > > "current"
> > > >> > > > > > > to
> > > >> > > > > > > > > > pair
> > > >> > > > > > > > > > >> > with
> > > >> > > > > > > > > > >> > > > the leaderId. I will update the wiki
> page.
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > Hmm, it's kind of weird to bump up the
> leader
> > > >> epoch
> > > >> > > > before
> > > >> > > > > > the
> > > >> > > > > > > > new
> > > >> > > > > > > > > > >> leader
> > > >> > > > > > > > > > >> > > is actually elected, right.
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > > >> > > > > > FetchQuorumRecords
> > > >> > > > > > > > API
> > > >> > > > > > > > > to
> > > >> > > > > > > > > > >> > > indicate
> > > >> > > > > > > > > > >> > > > > that the follower has fetched from an
> > > invalid
> > > >> > > offset
> > > >> > > > > and
> > > >> > > > > > > > > should
> > > >> > > > > > > > > > >> > > truncate
> > > >> > > > > > > > > > >> > > > to
> > > >> > > > > > > > > > >> > > > > the offset/epoch indicated in the
> > > response."
> > > >> > > > Observers
> > > >> > > > > > > can't
> > > >> > > > > > > > > > >> truncate
> > > >> > > > > > > > > > >> > > > their
> > > >> > > > > > > > > > >> > > > > logs. What should they do with
> > > >> > > OFFSET_OUT_OF_RANGE?
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > > I'm not sure if I understand your
> question?
> > > >> > > Observers
> > > >> > > > > > should
> > > >> > > > > > > > > still
> > > >> > > > > > > > > > >> be
> > > >> > > > > > > > > > >> > > able
> > > >> > > > > > > > > > >> > > > to truncate their logs as well.
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > Hmm, I thought only the quorum nodes have
> > local
> > > >> logs
> > > >> > > and
> > > >> > > > > > > > observers
> > > >> > > > > > > > > > >> don't?
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > > 107. "The leader will continue sending
> > > >> > > > BeginQuorumEpoch
> > > >> > > > > to
> > > >> > > > > > > > each
> > > >> > > > > > > > > > >> known
> > > >> > > > > > > > > > >> > > > voter
> > > >> > > > > > > > > > >> > > > > until it has received its endorsement."
> > If
> > > a
> > > >> > voter
> > > >> > > > is
> > > >> > > > > > down
> > > >> > > > > > > > > for a
> > > >> > > > > > > > > > >> long
> > > >> > > > > > > > > > >> > > > time,
> > > >> > > > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add
> > > >> > unnecessary
> > > >> > > > > > > overhead.
> > > >> > > > > > > > > > >> > Similarly,
> > > >> > > > > > > > > > >> > > > if a
> > > >> > > > > > > > > > >> > > > > follower stops sending
> > FetchQuorumRecords,
> > > >> does
> > > >> > > the
> > > >> > > > > > leader
> > > >> > > > > > > > > keep
> > > >> > > > > > > > > > >> > sending
> > > >> > > > > > > > > > >> > > > > BeginQuorumEpoch?
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a
> good
> > > >> point.
> > > >> > > The
> > > >> > > > > > > > > > >> > begin-quorum-epoch
> > > >> > > > > > > > > > >> > > > request is for voters to quickly get the
> > new
> > > >> > leader
> > > >> > > > > > > > information;
> > > >> > > > > > > > > > >> > however
> > > >> > > > > > > > > > >> > > > even if they do not get them they can
> still
> > > >> > > eventually
> > > >> > > > > > learn
> > > >> > > > > > > > > about
> > > >> > > > > > > > > > >> that
> > > >> > > > > > > > > > >> > > > from others via gossiping FindQuorum. I
> > think
> > > >> we
> > > >> > can
> > > >> > > > > > adjust
> > > >> > > > > > > > the
> > > >> > > > > > > > > > >> logic
> > > >> > > > > > > > > > >> > to
> > > >> > > > > > > > > > >> > > > e.g. exponential back-off or with a
> limited
> > > >> > > > num.retries.
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > Regarding FetchQuorumRecords: if the
> > follower
> > > >> > sends
> > > >> > > > > > > > > > >> FetchQuorumRecords
> > > >> > > > > > > > > > >> > > > already, it means that follower already
> > knows
> > > >> that
> > > >> > > the
> > > >> > > > > > > broker
> > > >> > > > > > > > is
> > > >> > > > > > > > > > the
> > > >> > > > > > > > > > >> > > > leader, and hence we can stop retrying
> > > >> > > > BeginQuorumEpoch;
> > > >> > > > > > > > however
> > > >> > > > > > > > > > it
> > > >> > > > > > > > > > >> is
> > > >> > > > > > > > > > >> > > > possible that after a follower sends
> > > >> > > > FetchQuorumRecords
> > > >> > > > > > > > already,
> > > >> > > > > > > > > > >> > suddenly
> > > >> > > > > > > > > > >> > > > it stops send it (possibly because it
> > learned
> > > >> > about
> > > >> > > a
> > > >> > > > > > higher
> > > >> > > > > > > > > epoch
> > > >> > > > > > > > > > >> > > leader),
> > > >> > > > > > > > > > >> > > > and hence this broker may be a "zombie"
> > > leader
> > > >> and
> > > >> > > we
> > > >> > > > > > > propose
> > > >> > > > > > > > to
> > > >> > > > > > > > > > use
> > > >> > > > > > > > > > >> > the
> > > >> > > > > > > > > > >> > > > fetch.timeout to let the leader to try to
> > > >> verify
> > > >> > if
> > > >> > > it
> > > >> > > > > has
> > > >> > > > > > > > > already
> > > >> > > > > > > > > > >> been
> > > >> > > > > > > > > > >> > > > stale.
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > It just seems that we should handle these
> two
> > > >> cases
> > > >> > > in a
> > > >> > > > > > > > > consistent
> > > >> > > > > > > > > > >> way?
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > Yes I agree, on the leader's side, the
> > > >> > > > FetchQuorumRecords
> > > >> > > > > > > from a
> > > >> > > > > > > > > > >> follower
> > > >> > > > > > > > > > >> > could mean that we no longer needs to send
> > > >> > > > BeginQuorumEpoch
> > > >> > > > > > > > anymore
> > > >> > > > > > > > > > ---
> > > >> > > > > > > > > > >> and
> > > >> > > > > > > > > > >> > it is already part of our current
> > implementations
> > > >> in
> > > >> > > > > > > > > > >> >
> > > >> > > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > > Thanks,
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > Jun
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > > > Jun
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM
> Guozhang
> > > >> Wang <
> > > >> > > > > > > > > > wangguoz@gmail.com
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > > > wrote:
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > > > > Hello Leonard,
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > Thanks for your comments, I'm relying
> > in
> > > >> line
> > > >> > > > below:
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang
> > > >> (Leonard)
> > > >> > > Ge
> > > >> > > > <
> > > >> > > > > > > > > > >> > wge@confluent.io>
> > > >> > > > > > > > > > >> > > > > > wrote:
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > > Hi Kafka developers,
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > > It's great to see this proposal and
> > it
> > > >> took
> > > >> > me
> > > >> > > > > some
> > > >> > > > > > > time
> > > >> > > > > > > > > to
> > > >> > > > > > > > > > >> > finish
> > > >> > > > > > > > > > >> > > > > > reading
> > > >> > > > > > > > > > >> > > > > > > it.
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > > And I have the following questions
> > > about
> > > >> the
> > > >> > > > > > Proposal:
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > >    - How do we plan to test this
> > design
> > > >> to
> > > >> > > > ensure
> > > >> > > > > > its
> > > >> > > > > > > > > > >> > correctness?
> > > >> > > > > > > > > > >> > > Or
> > > >> > > > > > > > > > >> > > > > > more
> > > >> > > > > > > > > > >> > > > > > >    broadly, how do we ensure that
> our
> > > new
> > > >> > > ‘pull’
> > > >> > > > > > based
> > > >> > > > > > > > > model
> > > >> > > > > > > > > > >> is
> > > >> > > > > > > > > > >> > > > > > functional
> > > >> > > > > > > > > > >> > > > > > > and
> > > >> > > > > > > > > > >> > > > > > >    correct given that it is
> different
> > > >> from
> > > >> > the
> > > >> > > > > > > original
> > > >> > > > > > > > > RAFT
> > > >> > > > > > > > > > >> > > > > > implementation
> > > >> > > > > > > > > > >> > > > > > >    which has formal proof of
> > > correctness?
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > We have two planned verifications on
> > the
> > > >> > > > correctness
> > > >> > > > > > and
> > > >> > > > > > > > > > >> liveness
> > > >> > > > > > > > > > >> > of
> > > >> > > > > > > > > > >> > > > the
> > > >> > > > > > > > > > >> > > > > > design. One is via model verification
> > > >> (TLA+)
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > https://github.com/guozhangwang/kafka-specification
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > Another is via the concurrent
> > simulation
> > > >> tests
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > >    - Have we considered any sensible
> > > >> defaults
> > > >> > > for
> > > >> > > > > the
> > > >> > > > > > > > > > >> > configuration,
> > > >> > > > > > > > > > >> > > > i.e.
> > > >> > > > > > > > > > >> > > > > > >    all the election timeout, fetch
> > time
> > > >> out,
> > > >> > > > etc.?
> > > >> > > > > > Or
> > > >> > > > > > > we
> > > >> > > > > > > > > > want
> > > >> > > > > > > > > > >> to
> > > >> > > > > > > > > > >> > > > leave
> > > >> > > > > > > > > > >> > > > > > > this to
> > > >> > > > > > > > > > >> > > > > > >    a later stage when we do the
> > > >> performance
> > > >> > > > > testing,
> > > >> > > > > > > > etc.
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > This is a good question, the reason
> we
> > > did
> > > >> not
> > > >> > > set
> > > >> > > > > any
> > > >> > > > > > > > > default
> > > >> > > > > > > > > > >> > values
> > > >> > > > > > > > > > >> > > > for
> > > >> > > > > > > > > > >> > > > > > the timeout configurations is that we
> > > >> think it
> > > >> > > may
> > > >> > > > > > take
> > > >> > > > > > > > some
> > > >> > > > > > > > > > >> > > > benchmarking
> > > >> > > > > > > > > > >> > > > > > experiments to get these defaults
> > right.
> > > >> Some
> > > >> > > > > > high-level
> > > >> > > > > > > > > > >> principles
> > > >> > > > > > > > > > >> > > to
> > > >> > > > > > > > > > >> > > > > > consider: 1) the fetch.timeout should
> > be
> > > >> > around
> > > >> > > > the
> > > >> > > > > > same
> > > >> > > > > > > > > scale
> > > >> > > > > > > > > > >> with
> > > >> > > > > > > > > > >> > > zk
> > > >> > > > > > > > > > >> > > > > > session timeout, which is now 18
> > seconds
> > > by
> > > >> > > > default
> > > >> > > > > --
> > > >> > > > > > > in
> > > >> > > > > > > > > > >> practice
> > > >> > > > > > > > > > >> > > > we've
> > > >> > > > > > > > > > >> > > > > > seen unstable networks having more
> than
> > > 10
> > > >> > secs
> > > >> > > of
> > > >> > > > > > > > transient
> > > >> > > > > > > > > > >> > > > > connectivity,
> > > >> > > > > > > > > > >> > > > > > 2) the election.timeout, however,
> > should
> > > be
> > > >> > > > smaller
> > > >> > > > > > than
> > > >> > > > > > > > the
> > > >> > > > > > > > > > >> fetch
> > > >> > > > > > > > > > >> > > > > timeout
> > > >> > > > > > > > > > >> > > > > > as is also suggested as a practical
> > > >> > optimization
> > > >> > > > in
> > > >> > > > > > > > > > literature:
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > >
> > > >> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > Some more discussions can be found
> > here:
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > >
> > > >> > > >
> https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > >    - Have we considered
> piggybacking
> > > >> > > > > > > `BeginQuorumEpoch`
> > > >> > > > > > > > > with
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > `
> > > >> > > > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be
> > > >> missing
> > > >> > > > > something
> > > >> > > > > > > > > obvious
> > > >> > > > > > > > > > >> but
> > > >> > > > > > > > > > >> > I
> > > >> > > > > > > > > > >> > > am
> > > >> > > > > > > > > > >> > > > > > just
> > > >> > > > > > > > > > >> > > > > > >    wondering why don’t we just use
> > the
> > > >> > > > > `FindQuorum`
> > > >> > > > > > > and
> > > >> > > > > > > > > > >> > > > > > > `FetchQuorumRecords`
> > > >> > > > > > > > > > >> > > > > > >    APIs and remove the
> > > `BeginQuorumEpoch`
> > > >> > API?
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is
> sent
> > > from
> > > >> > > leader
> > > >> > > > > ->
> > > >> > > > > > > > other
> > > >> > > > > > > > > > >> voter
> > > >> > > > > > > > > > >> > > > > > followers, while FindQuorum / Fetch
> are
> > > >> sent
> > > >> > > from
> > > >> > > > > > > follower
> > > >> > > > > > > > > to
> > > >> > > > > > > > > > >> > leader.
> > > >> > > > > > > > > > >> > > > > > Arguably one can eventually realize
> the
> > > new
> > > >> > > leader
> > > >> > > > > and
> > > >> > > > > > > > epoch
> > > >> > > > > > > > > > via
> > > >> > > > > > > > > > >> > > > > gossiping
> > > >> > > > > > > > > > >> > > > > > FindQuorum, but that could in
> practice
> > > >> > require a
> > > >> > > > > long
> > > >> > > > > > > > delay.
> > > >> > > > > > > > > > >> > Having a
> > > >> > > > > > > > > > >> > > > > > leader -> other voters request helps
> > the
> > > >> new
> > > >> > > > leader
> > > >> > > > > > > epoch
> > > >> > > > > > > > to
> > > >> > > > > > > > > > be
> > > >> > > > > > > > > > >> > > > > propagated
> > > >> > > > > > > > > > >> > > > > > faster under a pull model.
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > >    - And about the
> > `FetchQuorumRecords`
> > > >> > > response
> > > >> > > > > > > schema,
> > > >> > > > > > > > > in
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > > > > `Records`
> > > >> > > > > > > > > > >> > > > > > >    field of the response, is it
> just
> > > one
> > > >> > > record
> > > >> > > > or
> > > >> > > > > > all
> > > >> > > > > > > > the
> > > >> > > > > > > > > > >> > records
> > > >> > > > > > > > > > >> > > > > > starting
> > > >> > > > > > > > > > >> > > > > > >    from the FetchOffset? It seems a
> > lot
> > > >> more
> > > >> > > > > > efficient
> > > >> > > > > > > > if
> > > >> > > > > > > > > we
> > > >> > > > > > > > > > >> sent
> > > >> > > > > > > > > > >> > > all
> > > >> > > > > > > > > > >> > > > > the
> > > >> > > > > > > > > > >> > > > > > >    records during the bootstrapping
> > of
> > > >> the
> > > >> > > > > brokers.
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > Yes the fetching is batched:
> > FetchOffset
> > > is
> > > >> > just
> > > >> > > > the
> > > >> > > > > > > > > starting
> > > >> > > > > > > > > > >> > offset
> > > >> > > > > > > > > > >> > > of
> > > >> > > > > > > > > > >> > > > > the
> > > >> > > > > > > > > > >> > > > > > batch of records.
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > >    - Regarding the disruptive
> broker
> > > >> issues,
> > > >> > > > does
> > > >> > > > > > our
> > > >> > > > > > > > pull
> > > >> > > > > > > > > > >> based
> > > >> > > > > > > > > > >> > > > model
> > > >> > > > > > > > > > >> > > > > > >    suffer from it? If so, have we
> > > >> considered
> > > >> > > the
> > > >> > > > > > > > Pre-Vote
> > > >> > > > > > > > > > >> stage?
> > > >> > > > > > > > > > >> > If
> > > >> > > > > > > > > > >> > > > > not,
> > > >> > > > > > > > > > >> > > > > > > why?
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > The disruptive broker is stated in
> the
> > > >> > original
> > > >> > > > Raft
> > > >> > > > > > > paper
> > > >> > > > > > > > > > >> which is
> > > >> > > > > > > > > > >> > > the
> > > >> > > > > > > > > > >> > > > > > result of the push model design. Our
> > > >> analysis
> > > >> > > > showed
> > > >> > > > > > > that
> > > >> > > > > > > > > with
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > > pull
> > > >> > > > > > > > > > >> > > > > > model it is no longer an issue.
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > > Thanks a lot for putting this up,
> > and I
> > > >> hope
> > > >> > > > that
> > > >> > > > > my
> > > >> > > > > > > > > > questions
> > > >> > > > > > > > > > >> > can
> > > >> > > > > > > > > > >> > > be
> > > >> > > > > > > > > > >> > > > > of
> > > >> > > > > > > > > > >> > > > > > > some value to make this KIP better.
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > > Hope to hear from you soon!
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > > Best wishes,
> > > >> > > > > > > > > > >> > > > > > > Leonard
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM
> Colin
> > > >> > McCabe <
> > > >> > > > > > > > > > >> cmccabe@apache.org
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> > > > > wrote:
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > Hi Jason,
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > It's amazing to see this coming
> > > >> together
> > > >> > :)
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > I haven't had a chance to read in
> > > >> detail,
> > > >> > > but
> > > >> > > > I
> > > >> > > > > > read
> > > >> > > > > > > > the
> > > >> > > > > > > > > > >> > outline
> > > >> > > > > > > > > > >> > > > and
> > > >> > > > > > > > > > >> > > > > a
> > > >> > > > > > > > > > >> > > > > > > few
> > > >> > > > > > > > > > >> > > > > > > > things jumped out at me.
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > First, for every epoch that is 32
> > > bits
> > > >> > > rather
> > > >> > > > > than
> > > >> > > > > > > > 64, I
> > > >> > > > > > > > > > >> sort
> > > >> > > > > > > > > > >> > of
> > > >> > > > > > > > > > >> > > > > wonder
> > > >> > > > > > > > > > >> > > > > > > if
> > > >> > > > > > > > > > >> > > > > > > > that's a good long-term choice.
> I
> > > keep
> > > >> > > > reading
> > > >> > > > > > > about
> > > >> > > > > > > > > > stuff
> > > >> > > > > > > > > > >> > like
> > > >> > > > > > > > > > >> > > > > this:
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > >> > > > > > > > .
> > > >> > > > > > > > > > >> > > Obviously,
> > > >> > > > > > > > > > >> > > > > > that
> > > >> > > > > > > > > > >> > > > > > > > JIRA is about zxid, which
> > increments
> > > >> much
> > > >> > > > faster
> > > >> > > > > > > than
> > > >> > > > > > > > we
> > > >> > > > > > > > > > >> expect
> > > >> > > > > > > > > > >> > > > these
> > > >> > > > > > > > > > >> > > > > > > > leader epochs to, but it would
> > still
> > > be
> > > >> > good
> > > >> > > > to
> > > >> > > > > > see
> > > >> > > > > > > > some
> > > >> > > > > > > > > > >> rough
> > > >> > > > > > > > > > >> > > > > > > calculations
> > > >> > > > > > > > > > >> > > > > > > > about how long 32 bits (or
> really,
> > 31
> > > >> > bits)
> > > >> > > > will
> > > >> > > > > > > last
> > > >> > > > > > > > us
> > > >> > > > > > > > > > in
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > > > cases
> > > >> > > > > > > > > > >> > > > > > > where
> > > >> > > > > > > > > > >> > > > > > > > we're using it, and what the
> space
> > > >> savings
> > > >> > > > we're
> > > >> > > > > > > > getting
> > > >> > > > > > > > > > >> really
> > > >> > > > > > > > > > >> > > is.
> > > >> > > > > > > > > > >> > > > > It
> > > >> > > > > > > > > > >> > > > > > > > seems like in most cases the
> > tradeoff
> > > >> may
> > > >> > > not
> > > >> > > > be
> > > >> > > > > > > worth
> > > >> > > > > > > > > it?
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > Another thing I've been thinking
> > > about
> > > >> is
> > > >> > > how
> > > >> > > > we
> > > >> > > > > > do
> > > >> > > > > > > > > > >> > > > bootstrapping.  I
> > > >> > > > > > > > > > >> > > > > > > > would prefer to be in a world
> where
> > > >> > > > formatting a
> > > >> > > > > > new
> > > >> > > > > > > > > Kafka
> > > >> > > > > > > > > > >> node
> > > >> > > > > > > > > > >> > > > was a
> > > >> > > > > > > > > > >> > > > > > > first
> > > >> > > > > > > > > > >> > > > > > > > class operation explicitly
> > initiated
> > > by
> > > >> > the
> > > >> > > > > admin,
> > > >> > > > > > > > > rather
> > > >> > > > > > > > > > >> than
> > > >> > > > > > > > > > >> > > > > > something
> > > >> > > > > > > > > > >> > > > > > > > that happened implicitly when you
> > > >> started
> > > >> > up
> > > >> > > > the
> > > >> > > > > > > > broker
> > > >> > > > > > > > > > and
> > > >> > > > > > > > > > >> > > things
> > > >> > > > > > > > > > >> > > > > > > "looked
> > > >> > > > > > > > > > >> > > > > > > > blank."
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > The first problem is that things
> > can
> > > >> "look
> > > >> > > > > blank"
> > > >> > > > > > > > > > >> accidentally
> > > >> > > > > > > > > > >> > if
> > > >> > > > > > > > > > >> > > > the
> > > >> > > > > > > > > > >> > > > > > > > storage system is having a bad
> day.
> > > >> > Clearly
> > > >> > > > in
> > > >> > > > > > the
> > > >> > > > > > > > > > non-Raft
> > > >> > > > > > > > > > >> > > world,
> > > >> > > > > > > > > > >> > > > > > this
> > > >> > > > > > > > > > >> > > > > > > > leads to data loss if the broker
> > that
> > > >> is
> > > >> > > > > > (re)started
> > > >> > > > > > > > > this
> > > >> > > > > > > > > > >> way
> > > >> > > > > > > > > > >> > was
> > > >> > > > > > > > > > >> > > > the
> > > >> > > > > > > > > > >> > > > > > > > leader for some partitions.
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > The second problem is that we
> have
> > a
> > > >> bit
> > > >> > of
> > > >> > > a
> > > >> > > > > > > chicken
> > > >> > > > > > > > > and
> > > >> > > > > > > > > > >> egg
> > > >> > > > > > > > > > >> > > > problem
> > > >> > > > > > > > > > >> > > > > > > with
> > > >> > > > > > > > > > >> > > > > > > > certain configuration keys.  For
> > > >> example,
> > > >> > > > maybe
> > > >> > > > > > you
> > > >> > > > > > > > want
> > > >> > > > > > > > > > to
> > > >> > > > > > > > > > >> > > > configure
> > > >> > > > > > > > > > >> > > > > > > some
> > > >> > > > > > > > > > >> > > > > > > > connection security settings in
> > your
> > > >> > > cluster,
> > > >> > > > > but
> > > >> > > > > > > you
> > > >> > > > > > > > > > don't
> > > >> > > > > > > > > > >> > want
> > > >> > > > > > > > > > >> > > > them
> > > >> > > > > > > > > > >> > > > > > to
> > > >> > > > > > > > > > >> > > > > > > > ever be stored in a plaintext
> > config
> > > >> file.
> > > >> > > > (For
> > > >> > > > > > > > > example,
> > > >> > > > > > > > > > >> SCRAM
> > > >> > > > > > > > > > >> > > > > > > passwords,
> > > >> > > > > > > > > > >> > > > > > > > etc.)  You could use a broker API
> > to
> > > >> set
> > > >> > the
> > > >> > > > > > > > > > configuration,
> > > >> > > > > > > > > > >> but
> > > >> > > > > > > > > > >> > > > that
> > > >> > > > > > > > > > >> > > > > > > brings
> > > >> > > > > > > > > > >> > > > > > > > up the chicken and egg problem.
> > The
> > > >> > broker
> > > >> > > > > needs
> > > >> > > > > > to
> > > >> > > > > > > > be
> > > >> > > > > > > > > > >> > > configured
> > > >> > > > > > > > > > >> > > > to
> > > >> > > > > > > > > > >> > > > > > > know
> > > >> > > > > > > > > > >> > > > > > > > how to talk to you, but you need
> to
> > > >> > > configure
> > > >> > > > it
> > > >> > > > > > > > before
> > > >> > > > > > > > > > you
> > > >> > > > > > > > > > >> can
> > > >> > > > > > > > > > >> > > > talk
> > > >> > > > > > > > > > >> > > > > to
> > > >> > > > > > > > > > >> > > > > > > > it.  Using an external secret
> > manager
> > > >> like
> > > >> > > > Vault
> > > >> > > > > > is
> > > >> > > > > > > > one
> > > >> > > > > > > > > > way
> > > >> > > > > > > > > > >> to
> > > >> > > > > > > > > > >> > > > solve
> > > >> > > > > > > > > > >> > > > > > > this,
> > > >> > > > > > > > > > >> > > > > > > > but not everyone uses an external
> > > >> secret
> > > >> > > > > manager.
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > quorum.voters seems like a
> similar
> > > >> > > > configuration
> > > >> > > > > > > key.
> > > >> > > > > > > > > In
> > > >> > > > > > > > > > >> the
> > > >> > > > > > > > > > >> > > > current
> > > >> > > > > > > > > > >> > > > > > > KIP,
> > > >> > > > > > > > > > >> > > > > > > > this is only read if there is no
> > > other
> > > >> > > > > > configuration
> > > >> > > > > > > > > > >> specifying
> > > >> > > > > > > > > > >> > > the
> > > >> > > > > > > > > > >> > > > > > > quorum
> > > >> > > > > > > > > > >> > > > > > > > voter set.  If we had a
> kafka.mkfs
> > > >> > command,
> > > >> > > we
> > > >> > > > > > > > wouldn't
> > > >> > > > > > > > > > need
> > > >> > > > > > > > > > >> > this
> > > >> > > > > > > > > > >> > > > key
> > > >> > > > > > > > > > >> > > > > > > > because we could assume that
> there
> > > was
> > > >> > > always
> > > >> > > > > > quorum
> > > >> > > > > > > > > > >> > information
> > > >> > > > > > > > > > >> > > > > stored
> > > >> > > > > > > > > > >> > > > > > > > locally.
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > best,
> > > >> > > > > > > > > > >> > > > > > > > Colin
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44,
> > Jason
> > > >> > > > Gustafson
> > > >> > > > > > > wrote:
> > > >> > > > > > > > > > >> > > > > > > > > Hi All,
> > > >> > > > > > > > > > >> > > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > > I'd like to start a discussion
> on
> > > >> > KIP-595:
> > > >> > > > > > > > > > >> > > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > >> > > > > > > > > > >> > > > > > > > .
> > > >> > > > > > > > > > >> > > > > > > > > This proposal specifies a Raft
> > > >> protocol
> > > >> > to
> > > >> > > > > > > > ultimately
> > > >> > > > > > > > > > >> replace
> > > >> > > > > > > > > > >> > > > > > Zookeeper
> > > >> > > > > > > > > > >> > > > > > > > > as
> > > >> > > > > > > > > > >> > > > > > > > > documented in KIP-500. Please
> > take
> > > a
> > > >> > look
> > > >> > > > and
> > > >> > > > > > > share
> > > >> > > > > > > > > your
> > > >> > > > > > > > > > >> > > > thoughts.
> > > >> > > > > > > > > > >> > > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > > A few minor notes to set the
> > stage
> > > a
> > > >> > > little
> > > >> > > > > bit:
> > > >> > > > > > > > > > >> > > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > > - This KIP does not specify the
> > > >> > structure
> > > >> > > of
> > > >> > > > > the
> > > >> > > > > > > > > > messages
> > > >> > > > > > > > > > >> > used
> > > >> > > > > > > > > > >> > > to
> > > >> > > > > > > > > > >> > > > > > > > represent
> > > >> > > > > > > > > > >> > > > > > > > > metadata in Kafka, nor does it
> > > >> specify
> > > >> > the
> > > >> > > > > > > internal
> > > >> > > > > > > > > API
> > > >> > > > > > > > > > >> that
> > > >> > > > > > > > > > >> > > will
> > > >> > > > > > > > > > >> > > > > be
> > > >> > > > > > > > > > >> > > > > > > used
> > > >> > > > > > > > > > >> > > > > > > > > by the controller. Expect these
> > to
> > > >> come
> > > >> > in
> > > >> > > > > later
> > > >> > > > > > > > > > >> proposals.
> > > >> > > > > > > > > > >> > > Here
> > > >> > > > > > > > > > >> > > > we
> > > >> > > > > > > > > > >> > > > > > are
> > > >> > > > > > > > > > >> > > > > > > > > primarily concerned with the
> > > >> replication
> > > >> > > > > > protocol
> > > >> > > > > > > > and
> > > >> > > > > > > > > > >> basic
> > > >> > > > > > > > > > >> > > > > > operational
> > > >> > > > > > > > > > >> > > > > > > > > mechanics.
> > > >> > > > > > > > > > >> > > > > > > > > - We expect many details to
> > change
> > > >> as we
> > > >> > > get
> > > >> > > > > > > closer
> > > >> > > > > > > > to
> > > >> > > > > > > > > > >> > > > integration
> > > >> > > > > > > > > > >> > > > > > with
> > > >> > > > > > > > > > >> > > > > > > > > the controller. Any changes we
> > make
> > > >> will
> > > >> > > be
> > > >> > > > > made
> > > >> > > > > > > > > either
> > > >> > > > > > > > > > as
> > > >> > > > > > > > > > >> > > > > amendments
> > > >> > > > > > > > > > >> > > > > > > to
> > > >> > > > > > > > > > >> > > > > > > > > this KIP or, in the case of
> > larger
> > > >> > > changes,
> > > >> > > > as
> > > >> > > > > > new
> > > >> > > > > > > > > > >> proposals.
> > > >> > > > > > > > > > >> > > > > > > > > - We have a prototype
> > > implementation
> > > >> > > which I
> > > >> > > > > > will
> > > >> > > > > > > > put
> > > >> > > > > > > > > > >> online
> > > >> > > > > > > > > > >> > > > within
> > > >> > > > > > > > > > >> > > > > > the
> > > >> > > > > > > > > > >> > > > > > > > > next week which may help in
> > > >> > understanding
> > > >> > > > some
> > > >> > > > > > > > > details.
> > > >> > > > > > > > > > It
> > > >> > > > > > > > > > >> > has
> > > >> > > > > > > > > > >> > > > > > > diverged a
> > > >> > > > > > > > > > >> > > > > > > > > little bit from our proposal,
> so
> > I
> > > am
> > > >> > > > taking a
> > > >> > > > > > > > little
> > > >> > > > > > > > > > >> time to
> > > >> > > > > > > > > > >> > > > bring
> > > >> > > > > > > > > > >> > > > > > it
> > > >> > > > > > > > > > >> > > > > > > in
> > > >> > > > > > > > > > >> > > > > > > > > line. I'll post an update to
> this
> > > >> thread
> > > >> > > > when
> > > >> > > > > it
> > > >> > > > > > > is
> > > >> > > > > > > > > > >> available
> > > >> > > > > > > > > > >> > > for
> > > >> > > > > > > > > > >> > > > > > > review.
> > > >> > > > > > > > > > >> > > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > > Finally, I want to mention that
> > > this
> > > >> > > > proposal
> > > >> > > > > > was
> > > >> > > > > > > > > > drafted
> > > >> > > > > > > > > > >> by
> > > >> > > > > > > > > > >> > > > > myself,
> > > >> > > > > > > > > > >> > > > > > > > Boyang
> > > >> > > > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > >> > > > > > > > > > >> > > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > > > Thanks,
> > > >> > > > > > > > > > >> > > > > > > > > Jason
> > > >> > > > > > > > > > >> > > > > > > > >
> > > >> > > > > > > > > > >> > > > > > > >
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > > > --
> > > >> > > > > > > > > > >> > > > > > > Leonard Ge
> > > >> > > > > > > > > > >> > > > > > > Software Engineer Intern -
> Confluent
> > > >> > > > > > > > > > >> > > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > > > --
> > > >> > > > > > > > > > >> > > > > > -- Guozhang
> > > >> > > > > > > > > > >> > > > > >
> > > >> > > > > > > > > > >> > > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > > > --
> > > >> > > > > > > > > > >> > > > -- Guozhang
> > > >> > > > > > > > > > >> > > >
> > > >> > > > > > > > > > >> > >
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >> > --
> > > >> > > > > > > > > > >> > -- Guozhang
> > > >> > > > > > > > > > >> >
> > > >> > > > > > > > > > >>
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > > > --
> > > >> > > > > > > > > -- Guozhang
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > --
> > > >> > > > > > > -- Guozhang
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Jun,

I added a section on "Cluster Bootstrapping" which discusses clusterId
generation and the process through which brokers find the current leader.
The quick summary is that the first controller will be responsible for
generating the clusterId and persisting it in the metadata log. Before the
first leader has been elected, quorum APIs will skip clusterId validation.
This seems reasonable since this is primarily intended to prevent the
damage from misconfiguration after a cluster has been running for some
time. Upon startup, brokers begin by sending Fetch requests to find the
current leader. This will include the cluster.id from meta.properties if it
is present. The broker will shutdown immediately if it receives
INVALID_CLUSTER_ID from the Fetch response.

I also added some details about our testing strategy, which you asked about
previously.

Thanks,
Jason

On Mon, Jul 27, 2020 at 10:46 PM Boyang Chen <re...@gmail.com>
wrote:

> On Mon, Jul 27, 2020 at 4:58 AM Unmesh Joshi <un...@gmail.com>
> wrote:
>
> > Just checked etcd and zookeeper code, and both support leader to step
> down
> > as a follower to make sure there are no two leaders if the leader has
> been
> > disconnected from the majority of the followers
> > For etcd this is https://github.com/etcd-io/etcd/issues/3866
> > For Zookeeper its https://issues.apache.org/jira/browse/ZOOKEEPER-1699
> > I was just thinking if it would be difficult to implement in the Pull
> based
> > model, but I guess not. It is possibly the same way ISR list is managed
> > currently, if leader of the controller quorum loses majority of the
> > followers, it should step down and become follower, that way, telling
> > client in time that it was disconnected from the quorum, and not keep on
> > sending state metadata to clients.
> >
> > Thanks,
> > Unmesh
> >
> >
> > On Mon, Jul 27, 2020 at 9:31 AM Unmesh Joshi <un...@gmail.com>
> > wrote:
> >
> > > >>Could you clarify on this question? Which part of the raft group
> > doesn't
> > > >>know about leader dis-connection?
> > > The leader of the controller quorum is partitioned from the controller
> > > cluster, and a different leader is elected for the remaining controller
> > > cluster.
> >
> I see your concern. For KIP-595 implementation, since there is no regular
> heartbeats sent
> from the leader to the followers, we decided to piggy-back on the fetch
> timeout so that if the leader did not receive Fetch
> requests from a majority of the quorum for that amount of time, it would
> begin a new election and
> start sending VoteRequest to voter nodes in the cluster to understand the
> latest quorum. You could
> find more details in this section
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP595:ARaftProtocolfortheMetadataQuorum-Vote
> >
> .
>
>
> > > I think there are two things here,
> > > 1.  The old leader will not know if it's disconnected from the rest of
> > the
> > > controller quorum cluster unless it receives BeginQuorumEpoch from the
> > new
> > > leader. So it will keep on serving stale metadata to the clients
> > (Brokers,
> > > Producers and Consumers)
> > > 2. I assume, the Broker Leases will be managed on the controller quorum
> > > leader. This partitioned leader will keep on tracking broker leases it
> > has,
> > > while the new leader of the quorum will also start managing broker
> > leases.
> > > So while the quorum leader is partitioned, there will be two membership
> > > views of the kafka brokers managed on two leaders.
> > > Unless broker heartbeats are also replicated as part of the Raft log,
> > > there is no way to solve this?
> > > I know LogCabin implementation does replicate client heartbeats. I
> > suspect
> > > that the same issue is there in Zookeeper, which does not replicate
> > client
> > > Ping requests..
> > >
> > > Thanks,
> > > Unmesh
> > >
> > >
> > >
> > > On Mon, Jul 27, 2020 at 6:23 AM Boyang Chen <
> reluctanthero104@gmail.com>
> > > wrote:
> > >
> > >> Thanks for the questions Unmesh!
> > >>
> > >> On Sun, Jul 26, 2020 at 6:18 AM Unmesh Joshi <un...@gmail.com>
> > >> wrote:
> > >>
> > >> > Hi,
> > >> >
> > >> > In the FetchRequest Handling, how to make sure we handle scenarios
> > where
> > >> > the leader might have been disconnected from the cluster, but
> doesn't
> > >> know
> > >> > yet?
> > >> >
> > >> Could you clarify on this question? Which part of the raft group
> doesn't
> > >> know about leader
> > >> dis-connection?
> > >>
> > >>
> > >> > As discussed in the Raft Thesis section 6.4, the linearizable
> > semantics
> > >> of
> > >> > read requests is implemented in LogCabin by sending heartbeat to
> > >> followers
> > >> > and waiting till the heartbeats are successful to make sure that the
> > >> leader
> > >> > is still the leader.
> > >> > I think for the controller quorum to make sure none of the consumers
> > get
> > >> > stale data, it's important to have linearizable semantics? In the
> pull
> > >> > based model, the leader will need to wait for heartbeats from the
> > >> followers
> > >> > before returning each fetch request from the consumer then? Or do we
> > >> need
> > >> > to introduce some other request?
> > >> > (Zookeeper does not have linearizable semantics for read requests,
> but
> > >> as
> > >> > of now all the kafka interactions are through writes and watches).
> > >> >
> > >> > This is a very good question. For our v1 implementation we are not
> > >> aiming
> > >> to guarantee linearizable read, which
> > >> would be considered as a follow-up effort. Note that today in Kafka
> > there
> > >> is no guarantee on the metadata freshness either,
> > >> so no regression is introduced.
> > >>
> > >>
> > >> > Thanks,
> > >> > Unmesh
> > >> >
> > >> > On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > Hi, Jason,
> > >> > >
> > >> > > Thanks for the reply.
> > >> > >
> > >> > > 101. Sounds good. Regarding clusterId, I am not sure storing it in
> > the
> > >> > > metadata log is enough. For example, the vote request includes
> > >> clusterId.
> > >> > > So, no one can vote until they know the clusterId. Also, it would
> be
> > >> > useful
> > >> > > to support the case when a voter completely loses its disk and
> needs
> > >> to
> > >> > > recover.
> > >> > >
> > >> > > 210. There is no longer a FindQuorum request. When a follower
> > >> restarts,
> > >> > how
> > >> > > does it discover the leader? Is that based on DescribeQuorum? It
> > >> would be
> > >> > > useful to document this.
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > > On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <
> jason@confluent.io
> > >
> > >> > > wrote:
> > >> > >
> > >> > > > Hi Jun,
> > >> > > >
> > >> > > > Thanks for the questions.
> > >> > > >
> > >> > > > 101. I am treating some of the bootstrapping problems as out of
> > the
> > >> > scope
> > >> > > > of this KIP. I am working on a separate proposal which addresses
> > >> > > > bootstrapping security credentials specifically. Here is a rough
> > >> sketch
> > >> > > of
> > >> > > > how I am seeing it:
> > >> > > >
> > >> > > > 1. Dynamic broker configurations including encrypted passwords
> > will
> > >> be
> > >> > > > persisted in the metadata log and cached in the broker's
> > >> > > `meta.properties`
> > >> > > > file.
> > >> > > > 2. We will provide a tool which allows users to directly
> override
> > >> the
> > >> > > > values in `meta.properties` without requiring access to the
> > quorum.
> > >> > This
> > >> > > > can be used to bootstrap the credentials of the voter set itself
> > >> before
> > >> > > the
> > >> > > > cluster has been started.
> > >> > > > 3. Some dynamic config changes will only be allowed when a
> broker
> > is
> > >> > > > online. For example, changing a truststore password dynamically
> > >> would
> > >> > > > prevent that broker from being able to start if it were offline
> > when
> > >> > the
> > >> > > > change was made.
> > >> > > > 4. I am still thinking a little bit about SCRAM credentials, but
> > >> most
> > >> > > > likely they will be handled with an approach similar to
> > >> > > `meta.properties`.
> > >> > > >
> > >> > > > 101.3 As for the question about `clusterId`, I think the way we
> > >> would
> > >> > do
> > >> > > > this is to have the first elected leader generate a UUID and
> write
> > >> it
> > >> > to
> > >> > > > the metadata log. Let me add some detail to the proposal about
> > this.
> > >> > > >
> > >> > > > A few additional answers below:
> > >> > > >
> > >> > > > 203. Yes, that is correct.
> > >> > > >
> > >> > > > 204. That is a good question. What happens in this case is that
> > all
> > >> > > voters
> > >> > > > advance their epoch to the one designated by the candidate even
> if
> > >> they
> > >> > > > reject its vote request. Assuming the candidate fails to be
> > elected,
> > >> > the
> > >> > > > election will be retried until a leader emerges.
> > >> > > >
> > >> > > > 205. I had some discussion with Colin offline about this
> problem.
> > I
> > >> > think
> > >> > > > the answer should be "yes," but it probably needs a little more
> > >> > thought.
> > >> > > > Handling JBOD failures is tricky. For an observer, we can
> > replicate
> > >> the
> > >> > > > metadata log from scratch safely in a new log dir. But if the
> log
> > >> dir
> > >> > of
> > >> > > a
> > >> > > > voter fails, I do not think it is generally safe to start from
> an
> > >> empty
> > >> > > > state.
> > >> > > >
> > >> > > > 206. Yes, that is discussed in KIP-631 I believe.
> > >> > > >
> > >> > > > 207. Good suggestion. I will work on this.
> > >> > > >
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Jason
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io>
> wrote:
> > >> > > >
> > >> > > > > Hi, Jason,
> > >> > > > >
> > >> > > > > Thanks for the updated KIP. Looks good overall. A few more
> > >> comments
> > >> > > > below.
> > >> > > > >
> > >> > > > > 101. I still don't see a section on bootstrapping related
> > issues.
> > >> It
> > >> > > > would
> > >> > > > > be useful to document if/how the following is supported.
> > >> > > > > 101.1 Currently, we support auto broker id generation. Is this
> > >> > > supported
> > >> > > > > for bootstrap brokers?
> > >> > > > > 101.2 As Colin mentioned, sometimes we may need to load the
> > >> security
> > >> > > > > credentials to be broker before it can be connected to. Could
> > you
> > >> > > > provide a
> > >> > > > > bit more detail on how this will work?
> > >> > > > > 101.3 Currently, we use ZK to generate clusterId on a new
> > cluster.
> > >> > With
> > >> > > > > Raft, how does every broker generate the same clusterId in a
> > >> > > distributed
> > >> > > > > way?
> > >> > > > >
> > >> > > > > 200. It would be useful to document if the various special
> > offsets
> > >> > (log
> > >> > > > > start offset, recovery point, HWM, etc) for the Raft log are
> > >> stored
> > >> > in
> > >> > > > the
> > >> > > > > same existing checkpoint files or not.
> > >> > > > > 200.1 Since the Raft log flushes every append, does that allow
> > us
> > >> to
> > >> > > > > recover from a recovery point within the active segment or do
> we
> > >> > still
> > >> > > > need
> > >> > > > > to scan the full segment including the recovery point? The
> > former
> > >> can
> > >> > > be
> > >> > > > > tricky since multiple records can fall into the same disk page
> > >> and a
> > >> > > > > subsequent flush may corrupt a page with previously flushed
> > >> records.
> > >> > > > >
> > >> > > > > 201. Configurations.
> > >> > > > > 201.1 How do the Raft brokers get security related configs for
> > >> inter
> > >> > > > broker
> > >> > > > > communication? Is that based on the existing
> > >> > > > > inter.broker.security.protocol?
> > >> > > > > 201.2 We have quorum.retry.backoff.max.ms and
> > >> > quorum.retry.backoff.ms,
> > >> > > > but
> > >> > > > > only quorum.election.backoff.max.ms. This seems a bit
> > >> inconsistent.
> > >> > > > >
> > >> > > > > 202. Metrics:
> > >> > > > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
> > >> > > OutboundQueueTimeMs:
> > >> > > > > Are those the same as existing totalTime, requestQueueTime,
> > >> > localTime,
> > >> > > > > responseQueueTime? Could we reuse the existing ones with the
> tag
> > >> > > > > request=[request-type]?
> > >> > > > > 202.2. Could you explain what InboundChannelSize and
> > >> > > OutboundChannelSize
> > >> > > > > are?
> > >> > > > > 202.3 ElectionLatencyMax/Avg: It seems that both should be
> > >> windowed?
> > >> > > > >
> > >> > > > > 203. Quorum State: I assume that LeaderId will be kept
> > >> consistently
> > >> > > with
> > >> > > > > LeaderEpoch. For example, if a follower transitions to
> candidate
> > >> and
> > >> > > > bumps
> > >> > > > > up LeaderEpoch, it will set leaderId to -1 and persist both in
> > the
> > >> > > Quorum
> > >> > > > > state file. Is that correct?
> > >> > > > >
> > >> > > > > 204. I was thinking about a corner case when a Raft broker is
> > >> > > partitioned
> > >> > > > > off. This broker will then be in a continuous loop of bumping
> up
> > >> the
> > >> > > > leader
> > >> > > > > epoch, but failing to get enough votes. When the partitioning
> is
> > >> > > removed,
> > >> > > > > this broker's high leader epoch will force a leader election.
> I
> > >> > assume
> > >> > > > > other Raft brokers can immediately advance their leader epoch
> > >> passing
> > >> > > the
> > >> > > > > already bumped epoch such that leader election won't be
> delayed.
> > >> Is
> > >> > > that
> > >> > > > > right?
> > >> > > > >
> > >> > > > > 205. In a JBOD setting, could we use the existing tool to move
> > the
> > >> > Raft
> > >> > > > log
> > >> > > > > from one disk to another?
> > >> > > > >
> > >> > > > > 206. The KIP doesn't mention the local metadata store derived
> > from
> > >> > the
> > >> > > > Raft
> > >> > > > > log. Will that be covered in a separate KIP?
> > >> > > > >
> > >> > > > > 207. Since this is a critical component. Could we add a
> section
> > on
> > >> > the
> > >> > > > > testing plan for correctness?
> > >> > > > >
> > >> > > > > 208. Performance. Do we plan to do group commit (e.g. buffer
> > >> pending
> > >> > > > > appends during a flush and then flush all accumulated pending
> > >> records
> > >> > > > > together in the next flush) for better throughput?
> > >> > > > >
> > >> > > > > 209. "the leader can actually defer fsync until it knows
> > >> > "quorum.size -
> > >> > > > 1"
> > >> > > > > has get to a certain entry offset." Why is that "quorum.size -
> > 1"
> > >> > > instead
> > >> > > > > of the majority of the quorum?
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <
> > >> jason@confluent.io>
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > > > Hi All,
> > >> > > > > >
> > >> > > > > > Just a quick update on the proposal. We have decided to move
> > >> quorum
> > >> > > > > > reassignment to a separate KIP:
> > >> > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > >> > > > > > .
> > >> > > > > > The way this ties into cluster bootstrapping is complicated,
> > so
> > >> we
> > >> > > felt
> > >> > > > > we
> > >> > > > > > needed a bit more time for validation. That leaves the core
> of
> > >> this
> > >> > > > > > proposal as quorum-based replication. If there are no
> further
> > >> > > comments,
> > >> > > > > we
> > >> > > > > > will plan to start a vote later this week.
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Jason
> > >> > > > > >
> > >> > > > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <
> > >> wangguoz@gmail.com
> > >> > >
> > >> > > > > wrote:
> > >> > > > > >
> > >> > > > > > > @Jun Rao <ju...@gmail.com>
> > >> > > > > > >
> > >> > > > > > > Regarding your comment about log compaction. After some
> > >> > deep-diving
> > >> > > > > into
> > >> > > > > > > this we've decided to propose a new snapshot-based log
> > >> cleaning
> > >> > > > > mechanism
> > >> > > > > > > which would be used to replace the current compaction
> > >> mechanism
> > >> > for
> > >> > > > > this
> > >> > > > > > > meta log. A new KIP will be proposed specifically for this
> > >> idea.
> > >> > > > > > >
> > >> > > > > > > All,
> > >> > > > > > >
> > >> > > > > > > I've updated the KIP wiki a bit updating one config "
> > >> > > > > > > election.jitter.max.ms"
> > >> > > > > > > to "election.backoff.max.ms" to make it more clear about
> > the
> > >> > > usage:
> > >> > > > > the
> > >> > > > > > > configured value will be the upper bound of the binary
> > >> > exponential
> > >> > > > > > backoff
> > >> > > > > > > time after a failed election, before starting a new one.
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > Guozhang
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> > >> > > > > reluctanthero104@gmail.com>
> > >> > > > > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Thanks for the suggestions Guozhang.
> > >> > > > > > > >
> > >> > > > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
> > >> > > wangguoz@gmail.com>
> > >> > > > > > > wrote:
> > >> > > > > > > >
> > >> > > > > > > > > Hello Boyang,
> > >> > > > > > > > >
> > >> > > > > > > > > Thanks for the updated information. A few questions
> > here:
> > >> > > > > > > > >
> > >> > > > > > > > > 1) Should the quorum-file also update to support
> > >> multi-raft?
> > >> > > > > > > > >
> > >> > > > > > > > > I'm neutral about this, as we don't know yet how the
> > >> > multi-raft
> > >> > > > > > modules
> > >> > > > > > > > would behave. If
> > >> > > > > > > > we have different threads operating different raft
> groups,
> > >> > > > > > consolidating
> > >> > > > > > > > the `checkpoint` files seems
> > >> > > > > > > > not reasonable. We could always add `multi-quorum-file`
> > >> later
> > >> > if
> > >> > > > > > > possible.
> > >> > > > > > > >
> > >> > > > > > > > 2) In the previous proposal, there's fields in the
> > >> > > > FetchQuorumRecords
> > >> > > > > > > like
> > >> > > > > > > > > latestDirtyOffset, is that dropped intentionally?
> > >> > > > > > > > >
> > >> > > > > > > > > I dropped the latestDirtyOffset since it is associated
> > >> with
> > >> > the
> > >> > > > log
> > >> > > > > > > > compaction discussion. This is beyond this KIP scope and
> > we
> > >> > could
> > >> > > > > > > > potentially get a separate KIP to talk about it.
> > >> > > > > > > >
> > >> > > > > > > >
> > >> > > > > > > > > 3) I think we also need to elaborate a bit more
> details
> > >> > > regarding
> > >> > > > > > when
> > >> > > > > > > to
> > >> > > > > > > > > send metadata request and discover-brokers; currently
> we
> > >> only
> > >> > > > > > discussed
> > >> > > > > > > > > during bootstrap how these requests would be sent. I
> > think
> > >> > the
> > >> > > > > > > following
> > >> > > > > > > > > scenarios would also need these requests
> > >> > > > > > > > >
> > >> > > > > > > > > 3.a) As long as a broker does not know the current
> > quorum
> > >> > > > > (including
> > >> > > > > > > the
> > >> > > > > > > > > leader and the voters), it should continue
> periodically
> > >> ask
> > >> > > other
> > >> > > > > > > brokers
> > >> > > > > > > > > via "metadata.
> > >> > > > > > > > > 3.b) As long as a broker does not know all the current
> > >> quorum
> > >> > > > > voter's
> > >> > > > > > > > > connections, it should continue periodically ask other
> > >> > brokers
> > >> > > > via
> > >> > > > > > > > > "discover-brokers".
> > >> > > > > > > > > 3.c) When the leader's fetch timeout elapsed, it
> should
> > >> send
> > >> > > > > metadata
> > >> > > > > > > > > request.
> > >> > > > > > > > >
> > >> > > > > > > > > Make sense, will add to the KIP.
> > >> > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > > > Guozhang
> > >> > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > >> > > > > > > reluctanthero104@gmail.com>
> > >> > > > > > > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > > Hey all,
> > >> > > > > > > > > >
> > >> > > > > > > > > > follow-up on the previous email, we made some more
> > >> updates:
> > >> > > > > > > > > >
> > >> > > > > > > > > > 1. The Alter/DescribeQuorum RPCs are also
> > re-structured
> > >> to
> > >> > > use
> > >> > > > > > > > > multi-raft.
> > >> > > > > > > > > >
> > >> > > > > > > > > > 2. We add observer status into the
> > >> DescribeQuorumResponse
> > >> > as
> > >> > > we
> > >> > > > > see
> > >> > > > > > > it
> > >> > > > > > > > > is a
> > >> > > > > > > > > > low hanging fruit which is very useful for user
> > >> debugging
> > >> > and
> > >> > > > > > > > > reassignment.
> > >> > > > > > > > > >
> > >> > > > > > > > > > 3. The FindQuorum RPC is replaced with
> DiscoverBrokers
> > >> RPC,
> > >> > > > which
> > >> > > > > > is
> > >> > > > > > > > > purely
> > >> > > > > > > > > > in charge of discovering broker connections in a
> > gossip
> > >> > > manner.
> > >> > > > > The
> > >> > > > > > > > > quorum
> > >> > > > > > > > > > leader discovery is piggy-back on the Metadata RPC
> for
> > >> the
> > >> > > > topic
> > >> > > > > > > > > partition
> > >> > > > > > > > > > leader, which in our case is the single metadata
> > >> partition
> > >> > > for
> > >> > > > > the
> > >> > > > > > > > > version
> > >> > > > > > > > > > one.
> > >> > > > > > > > > >
> > >> > > > > > > > > > Let me know if you have any questions.
> > >> > > > > > > > > >
> > >> > > > > > > > > > Boyang
> > >> > > > > > > > > >
> > >> > > > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > >> > > > > > > > reluctanthero104@gmail.com>
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > > > Hey all,
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Thanks for the great discussions so far. I'm
> posting
> > >> some
> > >> > > KIP
> > >> > > > > > > updates
> > >> > > > > > > > > > from
> > >> > > > > > > > > > > our working group discussion:
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > 1. We will be changing the core RPCs from
> > single-raft
> > >> API
> > >> > > to
> > >> > > > > > > > > multi-raft.
> > >> > > > > > > > > > > This means all protocols will be "batch" in the
> > first
> > >> > > > version,
> > >> > > > > > but
> > >> > > > > > > > the
> > >> > > > > > > > > > KIP
> > >> > > > > > > > > > > itself only illustrates the design for a single
> > >> metadata
> > >> > > > topic
> > >> > > > > > > > > partition.
> > >> > > > > > > > > > > The reason is to "keep the door open" for future
> > >> > extensions
> > >> > > > of
> > >> > > > > > this
> > >> > > > > > > > > piece
> > >> > > > > > > > > > > of module such as a sharded controller or general
> > >> quorum
> > >> > > > based
> > >> > > > > > > topic
> > >> > > > > > > > > > > replication, beyond the current Kafka replication
> > >> > protocol.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > 2. We will piggy-back on the current Kafka Fetch
> API
> > >> > > instead
> > >> > > > of
> > >> > > > > > > > > inventing
> > >> > > > > > > > > > > a new FetchQuorumRecords RPC. The motivation is
> > about
> > >> the
> > >> > > > same
> > >> > > > > as
> > >> > > > > > > #1
> > >> > > > > > > > as
> > >> > > > > > > > > > > well as making the integration work easier,
> instead
> > of
> > >> > > > letting
> > >> > > > > > two
> > >> > > > > > > > > > similar
> > >> > > > > > > > > > > RPCs diverge.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of only
> > >> > sending
> > >> > > > the
> > >> > > > > > > > request
> > >> > > > > > > > > to
> > >> > > > > > > > > > > the most caught-up voter, we shall broadcast the
> > >> > > information
> > >> > > > to
> > >> > > > > > all
> > >> > > > > > > > > > voters,
> > >> > > > > > > > > > > with a sorted voter list in descending order of
> > their
> > >> > > > > > corresponding
> > >> > > > > > > > > > > replicated offset. In this way, the top voter will
> > >> > become a
> > >> > > > > > > candidate
> > >> > > > > > > > > > > immediately, while the other voters shall wait for
> > an
> > >> > > > > exponential
> > >> > > > > > > > > > back-off
> > >> > > > > > > > > > > to trigger elections, which helps ensure the top
> > voter
> > >> > gets
> > >> > > > > > > elected,
> > >> > > > > > > > > and
> > >> > > > > > > > > > > the election eventually happens when the top voter
> > is
> > >> not
> > >> > > > > > > responsive.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Please see the updated KIP and post any questions
> or
> > >> > > concerns
> > >> > > > > on
> > >> > > > > > > the
> > >> > > > > > > > > > > mailing thread.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Boyang
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <
> > >> jun@confluent.io
> > >> > >
> > >> > > > > wrote:
> > >> > > > > > > > > > >
> > >> > > > > > > > > > >> Hi, Guozhang and Jason,
> > >> > > > > > > > > > >>
> > >> > > > > > > > > > >> Thanks for the reply. A couple of more replies.
> > >> > > > > > > > > > >>
> > >> > > > > > > > > > >> 102. Still not sure about this. How is the
> > tombstone
> > >> > issue
> > >> > > > > > > addressed
> > >> > > > > > > > > in
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> non-voter and the observer.  They can die at any
> > >> point
> > >> > and
> > >> > > > > > restart
> > >> > > > > > > > at
> > >> > > > > > > > > an
> > >> > > > > > > > > > >> arbitrary later time, and the advancing of the
> > >> > firstDirty
> > >> > > > > offset
> > >> > > > > > > and
> > >> > > > > > > > > the
> > >> > > > > > > > > > >> removal of the tombstone can happen
> independently.
> > >> > > > > > > > > > >>
> > >> > > > > > > > > > >> 106. I agree that it would be less confusing if
> we
> > >> used
> > >> > > > > "epoch"
> > >> > > > > > > > > instead
> > >> > > > > > > > > > of
> > >> > > > > > > > > > >> "leader epoch" consistently.
> > >> > > > > > > > > > >>
> > >> > > > > > > > > > >> Jun
> > >> > > > > > > > > > >>
> > >> > > > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > >> > > > > > wangguoz@gmail.com>
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > > >>
> > >> > > > > > > > > > >> > Thanks Jun. Further replies are in-lined.
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
> > >> > > jun@confluent.io
> > >> > > > >
> > >> > > > > > > wrote:
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > > Hi, Guozhang,
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > Thanks for the reply. A few more replies
> > inlined
> > >> > > below.
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang
> <
> > >> > > > > > > > wangguoz@gmail.com>
> > >> > > > > > > > > > >> wrote:
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > > Hello Jun,
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > Thanks for your comments! I'm replying
> inline
> > >> > below:
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> > >> > > > > jun@confluent.io
> > >> > > > > > >
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > > 101. Bootstrapping related issues.
> > >> > > > > > > > > > >> > > > > 101.1 Currently, we support auto broker
> id
> > >> > > > generation.
> > >> > > > > > Is
> > >> > > > > > > > this
> > >> > > > > > > > > > >> > > supported
> > >> > > > > > > > > > >> > > > > for bootstrap brokers?
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > The vote ids would just be the broker ids.
> > >> > > > > > > "bootstrap.servers"
> > >> > > > > > > > > > >> would be
> > >> > > > > > > > > > >> > > > similar to what client configs have today,
> > >> where
> > >> > > > > > > > "quorum.voters"
> > >> > > > > > > > > > >> would
> > >> > > > > > > > > > >> > be
> > >> > > > > > > > > > >> > > > pre-defined config values.
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > My question was on the auto generated broker
> > id.
> > >> > > > > Currently,
> > >> > > > > > > the
> > >> > > > > > > > > > broker
> > >> > > > > > > > > > >> > can
> > >> > > > > > > > > > >> > > choose to have its broker Id auto generated.
> > The
> > >> > > > > generation
> > >> > > > > > is
> > >> > > > > > > > > done
> > >> > > > > > > > > > >> > through
> > >> > > > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's
> > not
> > >> > clear
> > >> > > > how
> > >> > > > > > the
> > >> > > > > > > > > > broker
> > >> > > > > > > > > > >> id
> > >> > > > > > > > > > >> > is
> > >> > > > > > > > > > >> > > auto generated. "quorum.voters" also can't be
> > set
> > >> > > > > statically
> > >> > > > > > > if
> > >> > > > > > > > > > broker
> > >> > > > > > > > > > >> > ids
> > >> > > > > > > > > > >> > > are auto generated.
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > Jason has explained some ideas that we've
> > >> discussed
> > >> > so
> > >> > > > > far,
> > >> > > > > > > the
> > >> > > > > > > > > > >> reason we
> > >> > > > > > > > > > >> > intentional did not include them so far is that
> > we
> > >> > feel
> > >> > > it
> > >> > > > > is
> > >> > > > > > > > > out-side
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500
> > we
> > >> > > should
> > >> > > > > > > > definitely
> > >> > > > > > > > > > >> > address them though.
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > On the high-level, our belief is that "joining
> a
> > >> > quorum"
> > >> > > > and
> > >> > > > > > > > > "joining
> > >> > > > > > > > > > >> (or
> > >> > > > > > > > > > >> > more specifically, registering brokers in) the
> > >> > cluster"
> > >> > > > > would
> > >> > > > > > be
> > >> > > > > > > > > > >> > de-coupled a bit, where the former should be
> > >> completed
> > >> > > > > before
> > >> > > > > > we
> > >> > > > > > > > do
> > >> > > > > > > > > > the
> > >> > > > > > > > > > >> > latter. More specifically, assuming the quorum
> is
> > >> > > already
> > >> > > > up
> > >> > > > > > and
> > >> > > > > > > > > > >> running,
> > >> > > > > > > > > > >> > after the newly started broker found the leader
> > of
> > >> the
> > >> > > > > quorum
> > >> > > > > > it
> > >> > > > > > > > can
> > >> > > > > > > > > > >> send a
> > >> > > > > > > > > > >> > specific RegisterBroker request including its
> > >> > listener /
> > >> > > > > > > protocol
> > >> > > > > > > > /
> > >> > > > > > > > > > etc,
> > >> > > > > > > > > > >> > and upon handling it the leader can send back
> the
> > >> > > uniquely
> > >> > > > > > > > generated
> > >> > > > > > > > > > >> broker
> > >> > > > > > > > > > >> > id to the new broker, while also executing the
> > >> > > > > > "startNewBroker"
> > >> > > > > > > > > > >> callback as
> > >> > > > > > > > > > >> > the controller.
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > > > 102. Log compaction. One weak spot of log
> > >> > > compaction
> > >> > > > > is
> > >> > > > > > > for
> > >> > > > > > > > > the
> > >> > > > > > > > > > >> > > consumer
> > >> > > > > > > > > > >> > > > to
> > >> > > > > > > > > > >> > > > > deal with deletes. When a key is deleted,
> > >> it's
> > >> > > > > retained
> > >> > > > > > > as a
> > >> > > > > > > > > > >> > tombstone
> > >> > > > > > > > > > >> > > > > first and then physically removed. If a
> > >> client
> > >> > > > misses
> > >> > > > > > the
> > >> > > > > > > > > > >> tombstone
> > >> > > > > > > > > > >> > > > > (because it's physically removed), it may
> > >> not be
> > >> > > > able
> > >> > > > > to
> > >> > > > > > > > > update
> > >> > > > > > > > > > >> its
> > >> > > > > > > > > > >> > > > > metadata properly. The way we solve this
> in
> > >> > Kafka
> > >> > > is
> > >> > > > > > based
> > >> > > > > > > > on
> > >> > > > > > > > > a
> > >> > > > > > > > > > >> > > > > configuration (
> > >> log.cleaner.delete.retention.ms)
> > >> > > and
> > >> > > > > we
> > >> > > > > > > > > expect a
> > >> > > > > > > > > > >> > > consumer
> > >> > > > > > > > > > >> > > > > having seen an old key to finish reading
> > the
> > >> > > > deletion
> > >> > > > > > > > > tombstone
> > >> > > > > > > > > > >> > within
> > >> > > > > > > > > > >> > > > that
> > >> > > > > > > > > > >> > > > > time. There is no strong guarantee for
> that
> > >> > since
> > >> > > a
> > >> > > > > > broker
> > >> > > > > > > > > could
> > >> > > > > > > > > > >> be
> > >> > > > > > > > > > >> > > down
> > >> > > > > > > > > > >> > > > > for a long time. It would be better if we
> > can
> > >> > > have a
> > >> > > > > > more
> > >> > > > > > > > > > reliable
> > >> > > > > > > > > > >> > way
> > >> > > > > > > > > > >> > > of
> > >> > > > > > > > > > >> > > > > dealing with deletes.
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > We propose to capture this in the
> > >> > "FirstDirtyOffset"
> > >> > > > > field
> > >> > > > > > > of
> > >> > > > > > > > > the
> > >> > > > > > > > > > >> > quorum
> > >> > > > > > > > > > >> > > > record fetch response: the offset is the
> > >> maximum
> > >> > > > offset
> > >> > > > > > that
> > >> > > > > > > > log
> > >> > > > > > > > > > >> > > compaction
> > >> > > > > > > > > > >> > > > has reached up to. If the follower has
> > fetched
> > >> > > beyond
> > >> > > > > this
> > >> > > > > > > > > offset
> > >> > > > > > > > > > it
> > >> > > > > > > > > > >> > > means
> > >> > > > > > > > > > >> > > > itself is safe hence it has seen all
> records
> > >> up to
> > >> > > > that
> > >> > > > > > > > offset.
> > >> > > > > > > > > On
> > >> > > > > > > > > > >> > > getting
> > >> > > > > > > > > > >> > > > the response, the follower can then decide
> if
> > >> its
> > >> > > end
> > >> > > > > > offset
> > >> > > > > > > > > > >> actually
> > >> > > > > > > > > > >> > > below
> > >> > > > > > > > > > >> > > > that dirty offset (and hence may miss some
> > >> > > > tombstones).
> > >> > > > > If
> > >> > > > > > > > > that's
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > > case:
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > 1) Naively, it could re-bootstrap metadata
> > log
> > >> > from
> > >> > > > the
> > >> > > > > > very
> > >> > > > > > > > > > >> beginning
> > >> > > > > > > > > > >> > to
> > >> > > > > > > > > > >> > > > catch up.
> > >> > > > > > > > > > >> > > > 2) During that time, it would refrain
> itself
> > >> from
> > >> > > > > > answering
> > >> > > > > > > > > > >> > > MetadataRequest
> > >> > > > > > > > > > >> > > > from any clients.
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > I am not sure that the "FirstDirtyOffset"
> field
> > >> > fully
> > >> > > > > > > addresses
> > >> > > > > > > > > the
> > >> > > > > > > > > > >> > issue.
> > >> > > > > > > > > > >> > > Currently, the deletion tombstone is not
> > removed
> > >> > > > > immediately
> > >> > > > > > > > > after a
> > >> > > > > > > > > > >> > round
> > >> > > > > > > > > > >> > > of cleaning. It's removed after a delay in a
> > >> > > subsequent
> > >> > > > > > round
> > >> > > > > > > of
> > >> > > > > > > > > > >> > cleaning.
> > >> > > > > > > > > > >> > > Consider an example where a key insertion is
> at
> > >> > offset
> > >> > > > 200
> > >> > > > > > > and a
> > >> > > > > > > > > > >> deletion
> > >> > > > > > > > > > >> > > tombstone of the key is at 400. Initially,
> > >> > > > > FirstDirtyOffset
> > >> > > > > > is
> > >> > > > > > > > at
> > >> > > > > > > > > > >> 300. A
> > >> > > > > > > > > > >> > > follower/observer fetches from offset 0  and
> > >> fetches
> > >> > > the
> > >> > > > > key
> > >> > > > > > > at
> > >> > > > > > > > > > offset
> > >> > > > > > > > > > >> > 200.
> > >> > > > > > > > > > >> > > A few rounds of cleaning happen.
> > >> FirstDirtyOffset is
> > >> > > at
> > >> > > > > 500
> > >> > > > > > > and
> > >> > > > > > > > > the
> > >> > > > > > > > > > >> > > tombstone at 400 is physically removed. The
> > >> > > > > > follower/observer
> > >> > > > > > > > > > >> continues
> > >> > > > > > > > > > >> > the
> > >> > > > > > > > > > >> > > fetch, but misses offset 400. It catches all
> > the
> > >> way
> > >> > > to
> > >> > > > > > > > > > >> FirstDirtyOffset
> > >> > > > > > > > > > >> > > and declares its metadata as ready. However,
> > its
> > >> > > > metadata
> > >> > > > > > > could
> > >> > > > > > > > be
> > >> > > > > > > > > > >> stale
> > >> > > > > > > > > > >> > > since it actually misses the deletion of the
> > key.
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > Yeah good question, I should have put more
> > >> details
> > >> > in
> > >> > > my
> > >> > > > > > > > > explanation
> > >> > > > > > > > > > >> :)
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > The idea is that we will adjust the log
> > compaction
> > >> for
> > >> > > > this
> > >> > > > > > raft
> > >> > > > > > > > > based
> > >> > > > > > > > > > >> > metadata log: before more details to be
> > explained,
> > >> > since
> > >> > > > we
> > >> > > > > > have
> > >> > > > > > > > two
> > >> > > > > > > > > > >> types
> > >> > > > > > > > > > >> > of "watermarks" here, whereas in Kafka the
> > >> watermark
> > >> > > > > indicates
> > >> > > > > > > > where
> > >> > > > > > > > > > >> every
> > >> > > > > > > > > > >> > replica have replicated up to and in Raft the
> > >> > watermark
> > >> > > > > > > indicates
> > >> > > > > > > > > > where
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > majority of replicas (here only indicating
> voters
> > >> of
> > >> > the
> > >> > > > > > quorum,
> > >> > > > > > > > not
> > >> > > > > > > > > > >> > counting observers) have replicated up to,
> let's
> > >> call
> > >> > > them
> > >> > > > > > Kafka
> > >> > > > > > > > > > >> watermark
> > >> > > > > > > > > > >> > and Raft watermark. For this special log, we
> > would
> > >> > > > maintain
> > >> > > > > > both
> > >> > > > > > > > > > >> > watermarks.
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > When log compacting on the leader, we would
> only
> > >> > compact
> > >> > > > up
> > >> > > > > to
> > >> > > > > > > the
> > >> > > > > > > > > > Kafka
> > >> > > > > > > > > > >> > watermark, i.e. if there is at least one voter
> > who
> > >> > have
> > >> > > > not
> > >> > > > > > > > > replicated
> > >> > > > > > > > > > >> an
> > >> > > > > > > > > > >> > entry, it would not be compacted. The
> > >> "dirty-offset"
> > >> > is
> > >> > > > the
> > >> > > > > > > offset
> > >> > > > > > > > > > that
> > >> > > > > > > > > > >> > we've compacted up to and is communicated to
> > other
> > >> > > voters,
> > >> > > > > and
> > >> > > > > > > the
> > >> > > > > > > > > > other
> > >> > > > > > > > > > >> > voters would also compact up to this value ---
> > i.e.
> > >> > the
> > >> > > > > > > difference
> > >> > > > > > > > > > here
> > >> > > > > > > > > > >> is
> > >> > > > > > > > > > >> > that instead of letting each replica doing log
> > >> > > compaction
> > >> > > > > > > > > > independently,
> > >> > > > > > > > > > >> > we'll have the leader to decide upon which
> offset
> > >> to
> > >> > > > compact
> > >> > > > > > to,
> > >> > > > > > > > and
> > >> > > > > > > > > > >> > propagate this value to others to follow, in a
> > more
> > >> > > > > > coordinated
> > >> > > > > > > > > > manner.
> > >> > > > > > > > > > >> > Also note when there are new voters joining the
> > >> quorum
> > >> > > who
> > >> > > > > has
> > >> > > > > > > not
> > >> > > > > > > > > > >> > replicated up to the dirty-offset, of because
> of
> > >> other
> > >> > > > > issues
> > >> > > > > > > they
> > >> > > > > > > > > > >> > truncated their logs to below the dirty-offset,
> > >> they'd
> > >> > > > have
> > >> > > > > to
> > >> > > > > > > > > > >> re-bootstrap
> > >> > > > > > > > > > >> > from the beginning, and during this period of
> > time
> > >> the
> > >> > > > > leader
> > >> > > > > > > > > learned
> > >> > > > > > > > > > >> about
> > >> > > > > > > > > > >> > this lagging voter would not advance the
> > watermark
> > >> > (also
> > >> > > > it
> > >> > > > > > > would
> > >> > > > > > > > > not
> > >> > > > > > > > > > >> > decrement it), and hence not compacting either,
> > >> until
> > >> > > the
> > >> > > > > > > voter(s)
> > >> > > > > > > > > has
> > >> > > > > > > > > > >> > caught up to that dirty-offset.
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > So back to your example above, before the
> > bootstrap
> > >> > > voter
> > >> > > > > gets
> > >> > > > > > > to
> > >> > > > > > > > > 300
> > >> > > > > > > > > > no
> > >> > > > > > > > > > >> > log compaction would happen on the leader; and
> > >> until
> > >> > > later
> > >> > > > > > when
> > >> > > > > > > > the
> > >> > > > > > > > > > >> voter
> > >> > > > > > > > > > >> > have got to beyond 400 and hence replicated
> that
> > >> > > > tombstone,
> > >> > > > > > the
> > >> > > > > > > > log
> > >> > > > > > > > > > >> > compaction would possibly get to that tombstone
> > and
> > >> > > remove
> > >> > > > > it.
> > >> > > > > > > Say
> > >> > > > > > > > > > >> later it
> > >> > > > > > > > > > >> > the leader's log compaction reaches 500, it can
> > >> send
> > >> > > this
> > >> > > > > back
> > >> > > > > > > to
> > >> > > > > > > > > the
> > >> > > > > > > > > > >> voter
> > >> > > > > > > > > > >> > who can then also compact locally up to 500.
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > > > > 105. Quorum State: In addition to
> VotedId,
> > >> do we
> > >> > > > need
> > >> > > > > > the
> > >> > > > > > > > > epoch
> > >> > > > > > > > > > >> > > > > corresponding to VotedId? Over time, the
> > same
> > >> > > broker
> > >> > > > > Id
> > >> > > > > > > > could
> > >> > > > > > > > > be
> > >> > > > > > > > > > >> > voted
> > >> > > > > > > > > > >> > > in
> > >> > > > > > > > > > >> > > > > different generations with different
> epoch.
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > > Hmm, this is a good point. Originally I
> think
> > >> the
> > >> > > > > > > > "LeaderEpoch"
> > >> > > > > > > > > > >> field
> > >> > > > > > > > > > >> > in
> > >> > > > > > > > > > >> > > > that file is corresponding to the "latest
> > known
> > >> > > leader
> > >> > > > > > > epoch",
> > >> > > > > > > > > not
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > > > "current leader epoch". For example, if the
> > >> > current
> > >> > > > > epoch
> > >> > > > > > is
> > >> > > > > > > > N,
> > >> > > > > > > > > > and
> > >> > > > > > > > > > >> > then
> > >> > > > > > > > > > >> > > a
> > >> > > > > > > > > > >> > > > vote-request with epoch N+1 is received and
> > the
> > >> > > voter
> > >> > > > > > > granted
> > >> > > > > > > > > the
> > >> > > > > > > > > > >> vote
> > >> > > > > > > > > > >> > > for
> > >> > > > > > > > > > >> > > > it, then it means for this voter it knows
> the
> > >> > > "latest
> > >> > > > > > epoch"
> > >> > > > > > > > is
> > >> > > > > > > > > N
> > >> > > > > > > > > > +
> > >> > > > > > > > > > >> 1
> > >> > > > > > > > > > >> > > > although it is unknown if that sending
> > >> candidate
> > >> > > will
> > >> > > > > > indeed
> > >> > > > > > > > > > become
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > > new
> > >> > > > > > > > > > >> > > > leader (which would only be notified via
> > >> > > begin-quorum
> > >> > > > > > > > request).
> > >> > > > > > > > > > >> > However,
> > >> > > > > > > > > > >> > > > when persisting the quorum state, we would
> > >> encode
> > >> > > > > > > leader-epoch
> > >> > > > > > > > > to
> > >> > > > > > > > > > >> N+1,
> > >> > > > > > > > > > >> > > > while the leaderId to be the older leader.
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > But now thinking about this a bit more, I
> > feel
> > >> we
> > >> > > > should
> > >> > > > > > use
> > >> > > > > > > > two
> > >> > > > > > > > > > >> > separate
> > >> > > > > > > > > > >> > > > epochs, one for the "lates known" and one
> for
> > >> the
> > >> > > > > > "current"
> > >> > > > > > > to
> > >> > > > > > > > > > pair
> > >> > > > > > > > > > >> > with
> > >> > > > > > > > > > >> > > > the leaderId. I will update the wiki page.
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > Hmm, it's kind of weird to bump up the leader
> > >> epoch
> > >> > > > before
> > >> > > > > > the
> > >> > > > > > > > new
> > >> > > > > > > > > > >> leader
> > >> > > > > > > > > > >> > > is actually elected, right.
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > >> > > > > > FetchQuorumRecords
> > >> > > > > > > > API
> > >> > > > > > > > > to
> > >> > > > > > > > > > >> > > indicate
> > >> > > > > > > > > > >> > > > > that the follower has fetched from an
> > invalid
> > >> > > offset
> > >> > > > > and
> > >> > > > > > > > > should
> > >> > > > > > > > > > >> > > truncate
> > >> > > > > > > > > > >> > > > to
> > >> > > > > > > > > > >> > > > > the offset/epoch indicated in the
> > response."
> > >> > > > Observers
> > >> > > > > > > can't
> > >> > > > > > > > > > >> truncate
> > >> > > > > > > > > > >> > > > their
> > >> > > > > > > > > > >> > > > > logs. What should they do with
> > >> > > OFFSET_OUT_OF_RANGE?
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > > I'm not sure if I understand your question?
> > >> > > Observers
> > >> > > > > > should
> > >> > > > > > > > > still
> > >> > > > > > > > > > >> be
> > >> > > > > > > > > > >> > > able
> > >> > > > > > > > > > >> > > > to truncate their logs as well.
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > Hmm, I thought only the quorum nodes have
> local
> > >> logs
> > >> > > and
> > >> > > > > > > > observers
> > >> > > > > > > > > > >> don't?
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > > 107. "The leader will continue sending
> > >> > > > BeginQuorumEpoch
> > >> > > > > to
> > >> > > > > > > > each
> > >> > > > > > > > > > >> known
> > >> > > > > > > > > > >> > > > voter
> > >> > > > > > > > > > >> > > > > until it has received its endorsement."
> If
> > a
> > >> > voter
> > >> > > > is
> > >> > > > > > down
> > >> > > > > > > > > for a
> > >> > > > > > > > > > >> long
> > >> > > > > > > > > > >> > > > time,
> > >> > > > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add
> > >> > unnecessary
> > >> > > > > > > overhead.
> > >> > > > > > > > > > >> > Similarly,
> > >> > > > > > > > > > >> > > > if a
> > >> > > > > > > > > > >> > > > > follower stops sending
> FetchQuorumRecords,
> > >> does
> > >> > > the
> > >> > > > > > leader
> > >> > > > > > > > > keep
> > >> > > > > > > > > > >> > sending
> > >> > > > > > > > > > >> > > > > BeginQuorumEpoch?
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good
> > >> point.
> > >> > > The
> > >> > > > > > > > > > >> > begin-quorum-epoch
> > >> > > > > > > > > > >> > > > request is for voters to quickly get the
> new
> > >> > leader
> > >> > > > > > > > information;
> > >> > > > > > > > > > >> > however
> > >> > > > > > > > > > >> > > > even if they do not get them they can still
> > >> > > eventually
> > >> > > > > > learn
> > >> > > > > > > > > about
> > >> > > > > > > > > > >> that
> > >> > > > > > > > > > >> > > > from others via gossiping FindQuorum. I
> think
> > >> we
> > >> > can
> > >> > > > > > adjust
> > >> > > > > > > > the
> > >> > > > > > > > > > >> logic
> > >> > > > > > > > > > >> > to
> > >> > > > > > > > > > >> > > > e.g. exponential back-off or with a limited
> > >> > > > num.retries.
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > Regarding FetchQuorumRecords: if the
> follower
> > >> > sends
> > >> > > > > > > > > > >> FetchQuorumRecords
> > >> > > > > > > > > > >> > > > already, it means that follower already
> knows
> > >> that
> > >> > > the
> > >> > > > > > > broker
> > >> > > > > > > > is
> > >> > > > > > > > > > the
> > >> > > > > > > > > > >> > > > leader, and hence we can stop retrying
> > >> > > > BeginQuorumEpoch;
> > >> > > > > > > > however
> > >> > > > > > > > > > it
> > >> > > > > > > > > > >> is
> > >> > > > > > > > > > >> > > > possible that after a follower sends
> > >> > > > FetchQuorumRecords
> > >> > > > > > > > already,
> > >> > > > > > > > > > >> > suddenly
> > >> > > > > > > > > > >> > > > it stops send it (possibly because it
> learned
> > >> > about
> > >> > > a
> > >> > > > > > higher
> > >> > > > > > > > > epoch
> > >> > > > > > > > > > >> > > leader),
> > >> > > > > > > > > > >> > > > and hence this broker may be a "zombie"
> > leader
> > >> and
> > >> > > we
> > >> > > > > > > propose
> > >> > > > > > > > to
> > >> > > > > > > > > > use
> > >> > > > > > > > > > >> > the
> > >> > > > > > > > > > >> > > > fetch.timeout to let the leader to try to
> > >> verify
> > >> > if
> > >> > > it
> > >> > > > > has
> > >> > > > > > > > > already
> > >> > > > > > > > > > >> been
> > >> > > > > > > > > > >> > > > stale.
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > It just seems that we should handle these two
> > >> cases
> > >> > > in a
> > >> > > > > > > > > consistent
> > >> > > > > > > > > > >> way?
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > Yes I agree, on the leader's side, the
> > >> > > > FetchQuorumRecords
> > >> > > > > > > from a
> > >> > > > > > > > > > >> follower
> > >> > > > > > > > > > >> > could mean that we no longer needs to send
> > >> > > > BeginQuorumEpoch
> > >> > > > > > > > anymore
> > >> > > > > > > > > > ---
> > >> > > > > > > > > > >> and
> > >> > > > > > > > > > >> > it is already part of our current
> implementations
> > >> in
> > >> > > > > > > > > > >> >
> > >> > > https://github.com/confluentinc/kafka/commits/kafka-raft
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > > Thanks,
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > Jun
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > > > Jun
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang
> > >> Wang <
> > >> > > > > > > > > > wangguoz@gmail.com
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > > > wrote:
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > > > > Hello Leonard,
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > Thanks for your comments, I'm relying
> in
> > >> line
> > >> > > > below:
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang
> > >> (Leonard)
> > >> > > Ge
> > >> > > > <
> > >> > > > > > > > > > >> > wge@confluent.io>
> > >> > > > > > > > > > >> > > > > > wrote:
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > > Hi Kafka developers,
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > > It's great to see this proposal and
> it
> > >> took
> > >> > me
> > >> > > > > some
> > >> > > > > > > time
> > >> > > > > > > > > to
> > >> > > > > > > > > > >> > finish
> > >> > > > > > > > > > >> > > > > > reading
> > >> > > > > > > > > > >> > > > > > > it.
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > > And I have the following questions
> > about
> > >> the
> > >> > > > > > Proposal:
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > >    - How do we plan to test this
> design
> > >> to
> > >> > > > ensure
> > >> > > > > > its
> > >> > > > > > > > > > >> > correctness?
> > >> > > > > > > > > > >> > > Or
> > >> > > > > > > > > > >> > > > > > more
> > >> > > > > > > > > > >> > > > > > >    broadly, how do we ensure that our
> > new
> > >> > > ‘pull’
> > >> > > > > > based
> > >> > > > > > > > > model
> > >> > > > > > > > > > >> is
> > >> > > > > > > > > > >> > > > > > functional
> > >> > > > > > > > > > >> > > > > > > and
> > >> > > > > > > > > > >> > > > > > >    correct given that it is different
> > >> from
> > >> > the
> > >> > > > > > > original
> > >> > > > > > > > > RAFT
> > >> > > > > > > > > > >> > > > > > implementation
> > >> > > > > > > > > > >> > > > > > >    which has formal proof of
> > correctness?
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > We have two planned verifications on
> the
> > >> > > > correctness
> > >> > > > > > and
> > >> > > > > > > > > > >> liveness
> > >> > > > > > > > > > >> > of
> > >> > > > > > > > > > >> > > > the
> > >> > > > > > > > > > >> > > > > > design. One is via model verification
> > >> (TLA+)
> > >> > > > > > > > > > >> > > > > >
> > >> > > > https://github.com/guozhangwang/kafka-specification
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > Another is via the concurrent
> simulation
> > >> tests
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >>
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > >    - Have we considered any sensible
> > >> defaults
> > >> > > for
> > >> > > > > the
> > >> > > > > > > > > > >> > configuration,
> > >> > > > > > > > > > >> > > > i.e.
> > >> > > > > > > > > > >> > > > > > >    all the election timeout, fetch
> time
> > >> out,
> > >> > > > etc.?
> > >> > > > > > Or
> > >> > > > > > > we
> > >> > > > > > > > > > want
> > >> > > > > > > > > > >> to
> > >> > > > > > > > > > >> > > > leave
> > >> > > > > > > > > > >> > > > > > > this to
> > >> > > > > > > > > > >> > > > > > >    a later stage when we do the
> > >> performance
> > >> > > > > testing,
> > >> > > > > > > > etc.
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > This is a good question, the reason we
> > did
> > >> not
> > >> > > set
> > >> > > > > any
> > >> > > > > > > > > default
> > >> > > > > > > > > > >> > values
> > >> > > > > > > > > > >> > > > for
> > >> > > > > > > > > > >> > > > > > the timeout configurations is that we
> > >> think it
> > >> > > may
> > >> > > > > > take
> > >> > > > > > > > some
> > >> > > > > > > > > > >> > > > benchmarking
> > >> > > > > > > > > > >> > > > > > experiments to get these defaults
> right.
> > >> Some
> > >> > > > > > high-level
> > >> > > > > > > > > > >> principles
> > >> > > > > > > > > > >> > > to
> > >> > > > > > > > > > >> > > > > > consider: 1) the fetch.timeout should
> be
> > >> > around
> > >> > > > the
> > >> > > > > > same
> > >> > > > > > > > > scale
> > >> > > > > > > > > > >> with
> > >> > > > > > > > > > >> > > zk
> > >> > > > > > > > > > >> > > > > > session timeout, which is now 18
> seconds
> > by
> > >> > > > default
> > >> > > > > --
> > >> > > > > > > in
> > >> > > > > > > > > > >> practice
> > >> > > > > > > > > > >> > > > we've
> > >> > > > > > > > > > >> > > > > > seen unstable networks having more than
> > 10
> > >> > secs
> > >> > > of
> > >> > > > > > > > transient
> > >> > > > > > > > > > >> > > > > connectivity,
> > >> > > > > > > > > > >> > > > > > 2) the election.timeout, however,
> should
> > be
> > >> > > > smaller
> > >> > > > > > than
> > >> > > > > > > > the
> > >> > > > > > > > > > >> fetch
> > >> > > > > > > > > > >> > > > > timeout
> > >> > > > > > > > > > >> > > > > > as is also suggested as a practical
> > >> > optimization
> > >> > > > in
> > >> > > > > > > > > > literature:
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > >
> > >> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > Some more discussions can be found
> here:
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > >
> > >> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > >    - Have we considered piggybacking
> > >> > > > > > > `BeginQuorumEpoch`
> > >> > > > > > > > > with
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > `
> > >> > > > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be
> > >> missing
> > >> > > > > something
> > >> > > > > > > > > obvious
> > >> > > > > > > > > > >> but
> > >> > > > > > > > > > >> > I
> > >> > > > > > > > > > >> > > am
> > >> > > > > > > > > > >> > > > > > just
> > >> > > > > > > > > > >> > > > > > >    wondering why don’t we just use
> the
> > >> > > > > `FindQuorum`
> > >> > > > > > > and
> > >> > > > > > > > > > >> > > > > > > `FetchQuorumRecords`
> > >> > > > > > > > > > >> > > > > > >    APIs and remove the
> > `BeginQuorumEpoch`
> > >> > API?
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent
> > from
> > >> > > leader
> > >> > > > > ->
> > >> > > > > > > > other
> > >> > > > > > > > > > >> voter
> > >> > > > > > > > > > >> > > > > > followers, while FindQuorum / Fetch are
> > >> sent
> > >> > > from
> > >> > > > > > > follower
> > >> > > > > > > > > to
> > >> > > > > > > > > > >> > leader.
> > >> > > > > > > > > > >> > > > > > Arguably one can eventually realize the
> > new
> > >> > > leader
> > >> > > > > and
> > >> > > > > > > > epoch
> > >> > > > > > > > > > via
> > >> > > > > > > > > > >> > > > > gossiping
> > >> > > > > > > > > > >> > > > > > FindQuorum, but that could in practice
> > >> > require a
> > >> > > > > long
> > >> > > > > > > > delay.
> > >> > > > > > > > > > >> > Having a
> > >> > > > > > > > > > >> > > > > > leader -> other voters request helps
> the
> > >> new
> > >> > > > leader
> > >> > > > > > > epoch
> > >> > > > > > > > to
> > >> > > > > > > > > > be
> > >> > > > > > > > > > >> > > > > propagated
> > >> > > > > > > > > > >> > > > > > faster under a pull model.
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > >    - And about the
> `FetchQuorumRecords`
> > >> > > response
> > >> > > > > > > schema,
> > >> > > > > > > > > in
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > > > > `Records`
> > >> > > > > > > > > > >> > > > > > >    field of the response, is it just
> > one
> > >> > > record
> > >> > > > or
> > >> > > > > > all
> > >> > > > > > > > the
> > >> > > > > > > > > > >> > records
> > >> > > > > > > > > > >> > > > > > starting
> > >> > > > > > > > > > >> > > > > > >    from the FetchOffset? It seems a
> lot
> > >> more
> > >> > > > > > efficient
> > >> > > > > > > > if
> > >> > > > > > > > > we
> > >> > > > > > > > > > >> sent
> > >> > > > > > > > > > >> > > all
> > >> > > > > > > > > > >> > > > > the
> > >> > > > > > > > > > >> > > > > > >    records during the bootstrapping
> of
> > >> the
> > >> > > > > brokers.
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > Yes the fetching is batched:
> FetchOffset
> > is
> > >> > just
> > >> > > > the
> > >> > > > > > > > > starting
> > >> > > > > > > > > > >> > offset
> > >> > > > > > > > > > >> > > of
> > >> > > > > > > > > > >> > > > > the
> > >> > > > > > > > > > >> > > > > > batch of records.
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > >    - Regarding the disruptive broker
> > >> issues,
> > >> > > > does
> > >> > > > > > our
> > >> > > > > > > > pull
> > >> > > > > > > > > > >> based
> > >> > > > > > > > > > >> > > > model
> > >> > > > > > > > > > >> > > > > > >    suffer from it? If so, have we
> > >> considered
> > >> > > the
> > >> > > > > > > > Pre-Vote
> > >> > > > > > > > > > >> stage?
> > >> > > > > > > > > > >> > If
> > >> > > > > > > > > > >> > > > > not,
> > >> > > > > > > > > > >> > > > > > > why?
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > The disruptive broker is stated in the
> > >> > original
> > >> > > > Raft
> > >> > > > > > > paper
> > >> > > > > > > > > > >> which is
> > >> > > > > > > > > > >> > > the
> > >> > > > > > > > > > >> > > > > > result of the push model design. Our
> > >> analysis
> > >> > > > showed
> > >> > > > > > > that
> > >> > > > > > > > > with
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > > pull
> > >> > > > > > > > > > >> > > > > > model it is no longer an issue.
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > > Thanks a lot for putting this up,
> and I
> > >> hope
> > >> > > > that
> > >> > > > > my
> > >> > > > > > > > > > questions
> > >> > > > > > > > > > >> > can
> > >> > > > > > > > > > >> > > be
> > >> > > > > > > > > > >> > > > > of
> > >> > > > > > > > > > >> > > > > > > some value to make this KIP better.
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > > Hope to hear from you soon!
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > > Best wishes,
> > >> > > > > > > > > > >> > > > > > > Leonard
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin
> > >> > McCabe <
> > >> > > > > > > > > > >> cmccabe@apache.org
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> > > > > wrote:
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > > > Hi Jason,
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > It's amazing to see this coming
> > >> together
> > >> > :)
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > I haven't had a chance to read in
> > >> detail,
> > >> > > but
> > >> > > > I
> > >> > > > > > read
> > >> > > > > > > > the
> > >> > > > > > > > > > >> > outline
> > >> > > > > > > > > > >> > > > and
> > >> > > > > > > > > > >> > > > > a
> > >> > > > > > > > > > >> > > > > > > few
> > >> > > > > > > > > > >> > > > > > > > things jumped out at me.
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > First, for every epoch that is 32
> > bits
> > >> > > rather
> > >> > > > > than
> > >> > > > > > > > 64, I
> > >> > > > > > > > > > >> sort
> > >> > > > > > > > > > >> > of
> > >> > > > > > > > > > >> > > > > wonder
> > >> > > > > > > > > > >> > > > > > > if
> > >> > > > > > > > > > >> > > > > > > > that's a good long-term choice.  I
> > keep
> > >> > > > reading
> > >> > > > > > > about
> > >> > > > > > > > > > stuff
> > >> > > > > > > > > > >> > like
> > >> > > > > > > > > > >> > > > > this:
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > >> > > > > > > > .
> > >> > > > > > > > > > >> > > Obviously,
> > >> > > > > > > > > > >> > > > > > that
> > >> > > > > > > > > > >> > > > > > > > JIRA is about zxid, which
> increments
> > >> much
> > >> > > > faster
> > >> > > > > > > than
> > >> > > > > > > > we
> > >> > > > > > > > > > >> expect
> > >> > > > > > > > > > >> > > > these
> > >> > > > > > > > > > >> > > > > > > > leader epochs to, but it would
> still
> > be
> > >> > good
> > >> > > > to
> > >> > > > > > see
> > >> > > > > > > > some
> > >> > > > > > > > > > >> rough
> > >> > > > > > > > > > >> > > > > > > calculations
> > >> > > > > > > > > > >> > > > > > > > about how long 32 bits (or really,
> 31
> > >> > bits)
> > >> > > > will
> > >> > > > > > > last
> > >> > > > > > > > us
> > >> > > > > > > > > > in
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > > > cases
> > >> > > > > > > > > > >> > > > > > > where
> > >> > > > > > > > > > >> > > > > > > > we're using it, and what the space
> > >> savings
> > >> > > > we're
> > >> > > > > > > > getting
> > >> > > > > > > > > > >> really
> > >> > > > > > > > > > >> > > is.
> > >> > > > > > > > > > >> > > > > It
> > >> > > > > > > > > > >> > > > > > > > seems like in most cases the
> tradeoff
> > >> may
> > >> > > not
> > >> > > > be
> > >> > > > > > > worth
> > >> > > > > > > > > it?
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > Another thing I've been thinking
> > about
> > >> is
> > >> > > how
> > >> > > > we
> > >> > > > > > do
> > >> > > > > > > > > > >> > > > bootstrapping.  I
> > >> > > > > > > > > > >> > > > > > > > would prefer to be in a world where
> > >> > > > formatting a
> > >> > > > > > new
> > >> > > > > > > > > Kafka
> > >> > > > > > > > > > >> node
> > >> > > > > > > > > > >> > > > was a
> > >> > > > > > > > > > >> > > > > > > first
> > >> > > > > > > > > > >> > > > > > > > class operation explicitly
> initiated
> > by
> > >> > the
> > >> > > > > admin,
> > >> > > > > > > > > rather
> > >> > > > > > > > > > >> than
> > >> > > > > > > > > > >> > > > > > something
> > >> > > > > > > > > > >> > > > > > > > that happened implicitly when you
> > >> started
> > >> > up
> > >> > > > the
> > >> > > > > > > > broker
> > >> > > > > > > > > > and
> > >> > > > > > > > > > >> > > things
> > >> > > > > > > > > > >> > > > > > > "looked
> > >> > > > > > > > > > >> > > > > > > > blank."
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > The first problem is that things
> can
> > >> "look
> > >> > > > > blank"
> > >> > > > > > > > > > >> accidentally
> > >> > > > > > > > > > >> > if
> > >> > > > > > > > > > >> > > > the
> > >> > > > > > > > > > >> > > > > > > > storage system is having a bad day.
> > >> > Clearly
> > >> > > > in
> > >> > > > > > the
> > >> > > > > > > > > > non-Raft
> > >> > > > > > > > > > >> > > world,
> > >> > > > > > > > > > >> > > > > > this
> > >> > > > > > > > > > >> > > > > > > > leads to data loss if the broker
> that
> > >> is
> > >> > > > > > (re)started
> > >> > > > > > > > > this
> > >> > > > > > > > > > >> way
> > >> > > > > > > > > > >> > was
> > >> > > > > > > > > > >> > > > the
> > >> > > > > > > > > > >> > > > > > > > leader for some partitions.
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > The second problem is that we have
> a
> > >> bit
> > >> > of
> > >> > > a
> > >> > > > > > > chicken
> > >> > > > > > > > > and
> > >> > > > > > > > > > >> egg
> > >> > > > > > > > > > >> > > > problem
> > >> > > > > > > > > > >> > > > > > > with
> > >> > > > > > > > > > >> > > > > > > > certain configuration keys.  For
> > >> example,
> > >> > > > maybe
> > >> > > > > > you
> > >> > > > > > > > want
> > >> > > > > > > > > > to
> > >> > > > > > > > > > >> > > > configure
> > >> > > > > > > > > > >> > > > > > > some
> > >> > > > > > > > > > >> > > > > > > > connection security settings in
> your
> > >> > > cluster,
> > >> > > > > but
> > >> > > > > > > you
> > >> > > > > > > > > > don't
> > >> > > > > > > > > > >> > want
> > >> > > > > > > > > > >> > > > them
> > >> > > > > > > > > > >> > > > > > to
> > >> > > > > > > > > > >> > > > > > > > ever be stored in a plaintext
> config
> > >> file.
> > >> > > > (For
> > >> > > > > > > > > example,
> > >> > > > > > > > > > >> SCRAM
> > >> > > > > > > > > > >> > > > > > > passwords,
> > >> > > > > > > > > > >> > > > > > > > etc.)  You could use a broker API
> to
> > >> set
> > >> > the
> > >> > > > > > > > > > configuration,
> > >> > > > > > > > > > >> but
> > >> > > > > > > > > > >> > > > that
> > >> > > > > > > > > > >> > > > > > > brings
> > >> > > > > > > > > > >> > > > > > > > up the chicken and egg problem.
> The
> > >> > broker
> > >> > > > > needs
> > >> > > > > > to
> > >> > > > > > > > be
> > >> > > > > > > > > > >> > > configured
> > >> > > > > > > > > > >> > > > to
> > >> > > > > > > > > > >> > > > > > > know
> > >> > > > > > > > > > >> > > > > > > > how to talk to you, but you need to
> > >> > > configure
> > >> > > > it
> > >> > > > > > > > before
> > >> > > > > > > > > > you
> > >> > > > > > > > > > >> can
> > >> > > > > > > > > > >> > > > talk
> > >> > > > > > > > > > >> > > > > to
> > >> > > > > > > > > > >> > > > > > > > it.  Using an external secret
> manager
> > >> like
> > >> > > > Vault
> > >> > > > > > is
> > >> > > > > > > > one
> > >> > > > > > > > > > way
> > >> > > > > > > > > > >> to
> > >> > > > > > > > > > >> > > > solve
> > >> > > > > > > > > > >> > > > > > > this,
> > >> > > > > > > > > > >> > > > > > > > but not everyone uses an external
> > >> secret
> > >> > > > > manager.
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > quorum.voters seems like a similar
> > >> > > > configuration
> > >> > > > > > > key.
> > >> > > > > > > > > In
> > >> > > > > > > > > > >> the
> > >> > > > > > > > > > >> > > > current
> > >> > > > > > > > > > >> > > > > > > KIP,
> > >> > > > > > > > > > >> > > > > > > > this is only read if there is no
> > other
> > >> > > > > > configuration
> > >> > > > > > > > > > >> specifying
> > >> > > > > > > > > > >> > > the
> > >> > > > > > > > > > >> > > > > > > quorum
> > >> > > > > > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs
> > >> > command,
> > >> > > we
> > >> > > > > > > > wouldn't
> > >> > > > > > > > > > need
> > >> > > > > > > > > > >> > this
> > >> > > > > > > > > > >> > > > key
> > >> > > > > > > > > > >> > > > > > > > because we could assume that there
> > was
> > >> > > always
> > >> > > > > > quorum
> > >> > > > > > > > > > >> > information
> > >> > > > > > > > > > >> > > > > stored
> > >> > > > > > > > > > >> > > > > > > > locally.
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > best,
> > >> > > > > > > > > > >> > > > > > > > Colin
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44,
> Jason
> > >> > > > Gustafson
> > >> > > > > > > wrote:
> > >> > > > > > > > > > >> > > > > > > > > Hi All,
> > >> > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > > I'd like to start a discussion on
> > >> > KIP-595:
> > >> > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >>
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > >> > > > > > > > > > >> > > > > > > > .
> > >> > > > > > > > > > >> > > > > > > > > This proposal specifies a Raft
> > >> protocol
> > >> > to
> > >> > > > > > > > ultimately
> > >> > > > > > > > > > >> replace
> > >> > > > > > > > > > >> > > > > > Zookeeper
> > >> > > > > > > > > > >> > > > > > > > > as
> > >> > > > > > > > > > >> > > > > > > > > documented in KIP-500. Please
> take
> > a
> > >> > look
> > >> > > > and
> > >> > > > > > > share
> > >> > > > > > > > > your
> > >> > > > > > > > > > >> > > > thoughts.
> > >> > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > > A few minor notes to set the
> stage
> > a
> > >> > > little
> > >> > > > > bit:
> > >> > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > > - This KIP does not specify the
> > >> > structure
> > >> > > of
> > >> > > > > the
> > >> > > > > > > > > > messages
> > >> > > > > > > > > > >> > used
> > >> > > > > > > > > > >> > > to
> > >> > > > > > > > > > >> > > > > > > > represent
> > >> > > > > > > > > > >> > > > > > > > > metadata in Kafka, nor does it
> > >> specify
> > >> > the
> > >> > > > > > > internal
> > >> > > > > > > > > API
> > >> > > > > > > > > > >> that
> > >> > > > > > > > > > >> > > will
> > >> > > > > > > > > > >> > > > > be
> > >> > > > > > > > > > >> > > > > > > used
> > >> > > > > > > > > > >> > > > > > > > > by the controller. Expect these
> to
> > >> come
> > >> > in
> > >> > > > > later
> > >> > > > > > > > > > >> proposals.
> > >> > > > > > > > > > >> > > Here
> > >> > > > > > > > > > >> > > > we
> > >> > > > > > > > > > >> > > > > > are
> > >> > > > > > > > > > >> > > > > > > > > primarily concerned with the
> > >> replication
> > >> > > > > > protocol
> > >> > > > > > > > and
> > >> > > > > > > > > > >> basic
> > >> > > > > > > > > > >> > > > > > operational
> > >> > > > > > > > > > >> > > > > > > > > mechanics.
> > >> > > > > > > > > > >> > > > > > > > > - We expect many details to
> change
> > >> as we
> > >> > > get
> > >> > > > > > > closer
> > >> > > > > > > > to
> > >> > > > > > > > > > >> > > > integration
> > >> > > > > > > > > > >> > > > > > with
> > >> > > > > > > > > > >> > > > > > > > > the controller. Any changes we
> make
> > >> will
> > >> > > be
> > >> > > > > made
> > >> > > > > > > > > either
> > >> > > > > > > > > > as
> > >> > > > > > > > > > >> > > > > amendments
> > >> > > > > > > > > > >> > > > > > > to
> > >> > > > > > > > > > >> > > > > > > > > this KIP or, in the case of
> larger
> > >> > > changes,
> > >> > > > as
> > >> > > > > > new
> > >> > > > > > > > > > >> proposals.
> > >> > > > > > > > > > >> > > > > > > > > - We have a prototype
> > implementation
> > >> > > which I
> > >> > > > > > will
> > >> > > > > > > > put
> > >> > > > > > > > > > >> online
> > >> > > > > > > > > > >> > > > within
> > >> > > > > > > > > > >> > > > > > the
> > >> > > > > > > > > > >> > > > > > > > > next week which may help in
> > >> > understanding
> > >> > > > some
> > >> > > > > > > > > details.
> > >> > > > > > > > > > It
> > >> > > > > > > > > > >> > has
> > >> > > > > > > > > > >> > > > > > > diverged a
> > >> > > > > > > > > > >> > > > > > > > > little bit from our proposal, so
> I
> > am
> > >> > > > taking a
> > >> > > > > > > > little
> > >> > > > > > > > > > >> time to
> > >> > > > > > > > > > >> > > > bring
> > >> > > > > > > > > > >> > > > > > it
> > >> > > > > > > > > > >> > > > > > > in
> > >> > > > > > > > > > >> > > > > > > > > line. I'll post an update to this
> > >> thread
> > >> > > > when
> > >> > > > > it
> > >> > > > > > > is
> > >> > > > > > > > > > >> available
> > >> > > > > > > > > > >> > > for
> > >> > > > > > > > > > >> > > > > > > review.
> > >> > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > > Finally, I want to mention that
> > this
> > >> > > > proposal
> > >> > > > > > was
> > >> > > > > > > > > > drafted
> > >> > > > > > > > > > >> by
> > >> > > > > > > > > > >> > > > > myself,
> > >> > > > > > > > > > >> > > > > > > > Boyang
> > >> > > > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > >> > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > >> > > > > > > > > Thanks,
> > >> > > > > > > > > > >> > > > > > > > > Jason
> > >> > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > > > --
> > >> > > > > > > > > > >> > > > > > > Leonard Ge
> > >> > > > > > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > >> > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > > > --
> > >> > > > > > > > > > >> > > > > > -- Guozhang
> > >> > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > >> > > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > > > --
> > >> > > > > > > > > > >> > > > -- Guozhang
> > >> > > > > > > > > > >> > > >
> > >> > > > > > > > > > >> > >
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >> > --
> > >> > > > > > > > > > >> > -- Guozhang
> > >> > > > > > > > > > >> >
> > >> > > > > > > > > > >>
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > > > --
> > >> > > > > > > > > -- Guozhang
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > --
> > >> > > > > > > -- Guozhang
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Boyang Chen <re...@gmail.com>.
On Mon, Jul 27, 2020 at 4:58 AM Unmesh Joshi <un...@gmail.com> wrote:

> Just checked etcd and zookeeper code, and both support leader to step down
> as a follower to make sure there are no two leaders if the leader has been
> disconnected from the majority of the followers
> For etcd this is https://github.com/etcd-io/etcd/issues/3866
> For Zookeeper its https://issues.apache.org/jira/browse/ZOOKEEPER-1699
> I was just thinking if it would be difficult to implement in the Pull based
> model, but I guess not. It is possibly the same way ISR list is managed
> currently, if leader of the controller quorum loses majority of the
> followers, it should step down and become follower, that way, telling
> client in time that it was disconnected from the quorum, and not keep on
> sending state metadata to clients.
>
> Thanks,
> Unmesh
>
>
> On Mon, Jul 27, 2020 at 9:31 AM Unmesh Joshi <un...@gmail.com>
> wrote:
>
> > >>Could you clarify on this question? Which part of the raft group
> doesn't
> > >>know about leader dis-connection?
> > The leader of the controller quorum is partitioned from the controller
> > cluster, and a different leader is elected for the remaining controller
> > cluster.
>
I see your concern. For KIP-595 implementation, since there is no regular
heartbeats sent
from the leader to the followers, we decided to piggy-back on the fetch
timeout so that if the leader did not receive Fetch
requests from a majority of the quorum for that amount of time, it would
begin a new election and
start sending VoteRequest to voter nodes in the cluster to understand the
latest quorum. You could
find more details in this section
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP595:ARaftProtocolfortheMetadataQuorum-Vote>
.


> > I think there are two things here,
> > 1.  The old leader will not know if it's disconnected from the rest of
> the
> > controller quorum cluster unless it receives BeginQuorumEpoch from the
> new
> > leader. So it will keep on serving stale metadata to the clients
> (Brokers,
> > Producers and Consumers)
> > 2. I assume, the Broker Leases will be managed on the controller quorum
> > leader. This partitioned leader will keep on tracking broker leases it
> has,
> > while the new leader of the quorum will also start managing broker
> leases.
> > So while the quorum leader is partitioned, there will be two membership
> > views of the kafka brokers managed on two leaders.
> > Unless broker heartbeats are also replicated as part of the Raft log,
> > there is no way to solve this?
> > I know LogCabin implementation does replicate client heartbeats. I
> suspect
> > that the same issue is there in Zookeeper, which does not replicate
> client
> > Ping requests..
> >
> > Thanks,
> > Unmesh
> >
> >
> >
> > On Mon, Jul 27, 2020 at 6:23 AM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> >> Thanks for the questions Unmesh!
> >>
> >> On Sun, Jul 26, 2020 at 6:18 AM Unmesh Joshi <un...@gmail.com>
> >> wrote:
> >>
> >> > Hi,
> >> >
> >> > In the FetchRequest Handling, how to make sure we handle scenarios
> where
> >> > the leader might have been disconnected from the cluster, but doesn't
> >> know
> >> > yet?
> >> >
> >> Could you clarify on this question? Which part of the raft group doesn't
> >> know about leader
> >> dis-connection?
> >>
> >>
> >> > As discussed in the Raft Thesis section 6.4, the linearizable
> semantics
> >> of
> >> > read requests is implemented in LogCabin by sending heartbeat to
> >> followers
> >> > and waiting till the heartbeats are successful to make sure that the
> >> leader
> >> > is still the leader.
> >> > I think for the controller quorum to make sure none of the consumers
> get
> >> > stale data, it's important to have linearizable semantics? In the pull
> >> > based model, the leader will need to wait for heartbeats from the
> >> followers
> >> > before returning each fetch request from the consumer then? Or do we
> >> need
> >> > to introduce some other request?
> >> > (Zookeeper does not have linearizable semantics for read requests, but
> >> as
> >> > of now all the kafka interactions are through writes and watches).
> >> >
> >> > This is a very good question. For our v1 implementation we are not
> >> aiming
> >> to guarantee linearizable read, which
> >> would be considered as a follow-up effort. Note that today in Kafka
> there
> >> is no guarantee on the metadata freshness either,
> >> so no regression is introduced.
> >>
> >>
> >> > Thanks,
> >> > Unmesh
> >> >
> >> > On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > Hi, Jason,
> >> > >
> >> > > Thanks for the reply.
> >> > >
> >> > > 101. Sounds good. Regarding clusterId, I am not sure storing it in
> the
> >> > > metadata log is enough. For example, the vote request includes
> >> clusterId.
> >> > > So, no one can vote until they know the clusterId. Also, it would be
> >> > useful
> >> > > to support the case when a voter completely loses its disk and needs
> >> to
> >> > > recover.
> >> > >
> >> > > 210. There is no longer a FindQuorum request. When a follower
> >> restarts,
> >> > how
> >> > > does it discover the leader? Is that based on DescribeQuorum? It
> >> would be
> >> > > useful to document this.
> >> > >
> >> > > Jun
> >> > >
> >> > > On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <jason@confluent.io
> >
> >> > > wrote:
> >> > >
> >> > > > Hi Jun,
> >> > > >
> >> > > > Thanks for the questions.
> >> > > >
> >> > > > 101. I am treating some of the bootstrapping problems as out of
> the
> >> > scope
> >> > > > of this KIP. I am working on a separate proposal which addresses
> >> > > > bootstrapping security credentials specifically. Here is a rough
> >> sketch
> >> > > of
> >> > > > how I am seeing it:
> >> > > >
> >> > > > 1. Dynamic broker configurations including encrypted passwords
> will
> >> be
> >> > > > persisted in the metadata log and cached in the broker's
> >> > > `meta.properties`
> >> > > > file.
> >> > > > 2. We will provide a tool which allows users to directly override
> >> the
> >> > > > values in `meta.properties` without requiring access to the
> quorum.
> >> > This
> >> > > > can be used to bootstrap the credentials of the voter set itself
> >> before
> >> > > the
> >> > > > cluster has been started.
> >> > > > 3. Some dynamic config changes will only be allowed when a broker
> is
> >> > > > online. For example, changing a truststore password dynamically
> >> would
> >> > > > prevent that broker from being able to start if it were offline
> when
> >> > the
> >> > > > change was made.
> >> > > > 4. I am still thinking a little bit about SCRAM credentials, but
> >> most
> >> > > > likely they will be handled with an approach similar to
> >> > > `meta.properties`.
> >> > > >
> >> > > > 101.3 As for the question about `clusterId`, I think the way we
> >> would
> >> > do
> >> > > > this is to have the first elected leader generate a UUID and write
> >> it
> >> > to
> >> > > > the metadata log. Let me add some detail to the proposal about
> this.
> >> > > >
> >> > > > A few additional answers below:
> >> > > >
> >> > > > 203. Yes, that is correct.
> >> > > >
> >> > > > 204. That is a good question. What happens in this case is that
> all
> >> > > voters
> >> > > > advance their epoch to the one designated by the candidate even if
> >> they
> >> > > > reject its vote request. Assuming the candidate fails to be
> elected,
> >> > the
> >> > > > election will be retried until a leader emerges.
> >> > > >
> >> > > > 205. I had some discussion with Colin offline about this problem.
> I
> >> > think
> >> > > > the answer should be "yes," but it probably needs a little more
> >> > thought.
> >> > > > Handling JBOD failures is tricky. For an observer, we can
> replicate
> >> the
> >> > > > metadata log from scratch safely in a new log dir. But if the log
> >> dir
> >> > of
> >> > > a
> >> > > > voter fails, I do not think it is generally safe to start from an
> >> empty
> >> > > > state.
> >> > > >
> >> > > > 206. Yes, that is discussed in KIP-631 I believe.
> >> > > >
> >> > > > 207. Good suggestion. I will work on this.
> >> > > >
> >> > > >
> >> > > > Thanks,
> >> > > > Jason
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:
> >> > > >
> >> > > > > Hi, Jason,
> >> > > > >
> >> > > > > Thanks for the updated KIP. Looks good overall. A few more
> >> comments
> >> > > > below.
> >> > > > >
> >> > > > > 101. I still don't see a section on bootstrapping related
> issues.
> >> It
> >> > > > would
> >> > > > > be useful to document if/how the following is supported.
> >> > > > > 101.1 Currently, we support auto broker id generation. Is this
> >> > > supported
> >> > > > > for bootstrap brokers?
> >> > > > > 101.2 As Colin mentioned, sometimes we may need to load the
> >> security
> >> > > > > credentials to be broker before it can be connected to. Could
> you
> >> > > > provide a
> >> > > > > bit more detail on how this will work?
> >> > > > > 101.3 Currently, we use ZK to generate clusterId on a new
> cluster.
> >> > With
> >> > > > > Raft, how does every broker generate the same clusterId in a
> >> > > distributed
> >> > > > > way?
> >> > > > >
> >> > > > > 200. It would be useful to document if the various special
> offsets
> >> > (log
> >> > > > > start offset, recovery point, HWM, etc) for the Raft log are
> >> stored
> >> > in
> >> > > > the
> >> > > > > same existing checkpoint files or not.
> >> > > > > 200.1 Since the Raft log flushes every append, does that allow
> us
> >> to
> >> > > > > recover from a recovery point within the active segment or do we
> >> > still
> >> > > > need
> >> > > > > to scan the full segment including the recovery point? The
> former
> >> can
> >> > > be
> >> > > > > tricky since multiple records can fall into the same disk page
> >> and a
> >> > > > > subsequent flush may corrupt a page with previously flushed
> >> records.
> >> > > > >
> >> > > > > 201. Configurations.
> >> > > > > 201.1 How do the Raft brokers get security related configs for
> >> inter
> >> > > > broker
> >> > > > > communication? Is that based on the existing
> >> > > > > inter.broker.security.protocol?
> >> > > > > 201.2 We have quorum.retry.backoff.max.ms and
> >> > quorum.retry.backoff.ms,
> >> > > > but
> >> > > > > only quorum.election.backoff.max.ms. This seems a bit
> >> inconsistent.
> >> > > > >
> >> > > > > 202. Metrics:
> >> > > > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
> >> > > OutboundQueueTimeMs:
> >> > > > > Are those the same as existing totalTime, requestQueueTime,
> >> > localTime,
> >> > > > > responseQueueTime? Could we reuse the existing ones with the tag
> >> > > > > request=[request-type]?
> >> > > > > 202.2. Could you explain what InboundChannelSize and
> >> > > OutboundChannelSize
> >> > > > > are?
> >> > > > > 202.3 ElectionLatencyMax/Avg: It seems that both should be
> >> windowed?
> >> > > > >
> >> > > > > 203. Quorum State: I assume that LeaderId will be kept
> >> consistently
> >> > > with
> >> > > > > LeaderEpoch. For example, if a follower transitions to candidate
> >> and
> >> > > > bumps
> >> > > > > up LeaderEpoch, it will set leaderId to -1 and persist both in
> the
> >> > > Quorum
> >> > > > > state file. Is that correct?
> >> > > > >
> >> > > > > 204. I was thinking about a corner case when a Raft broker is
> >> > > partitioned
> >> > > > > off. This broker will then be in a continuous loop of bumping up
> >> the
> >> > > > leader
> >> > > > > epoch, but failing to get enough votes. When the partitioning is
> >> > > removed,
> >> > > > > this broker's high leader epoch will force a leader election. I
> >> > assume
> >> > > > > other Raft brokers can immediately advance their leader epoch
> >> passing
> >> > > the
> >> > > > > already bumped epoch such that leader election won't be delayed.
> >> Is
> >> > > that
> >> > > > > right?
> >> > > > >
> >> > > > > 205. In a JBOD setting, could we use the existing tool to move
> the
> >> > Raft
> >> > > > log
> >> > > > > from one disk to another?
> >> > > > >
> >> > > > > 206. The KIP doesn't mention the local metadata store derived
> from
> >> > the
> >> > > > Raft
> >> > > > > log. Will that be covered in a separate KIP?
> >> > > > >
> >> > > > > 207. Since this is a critical component. Could we add a section
> on
> >> > the
> >> > > > > testing plan for correctness?
> >> > > > >
> >> > > > > 208. Performance. Do we plan to do group commit (e.g. buffer
> >> pending
> >> > > > > appends during a flush and then flush all accumulated pending
> >> records
> >> > > > > together in the next flush) for better throughput?
> >> > > > >
> >> > > > > 209. "the leader can actually defer fsync until it knows
> >> > "quorum.size -
> >> > > > 1"
> >> > > > > has get to a certain entry offset." Why is that "quorum.size -
> 1"
> >> > > instead
> >> > > > > of the majority of the quorum?
> >> > > > >
> >> > > > > Thanks,
> >> > > > >
> >> > > > > Jun
> >> > > > >
> >> > > > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <
> >> jason@confluent.io>
> >> > > > > wrote:
> >> > > > >
> >> > > > > > Hi All,
> >> > > > > >
> >> > > > > > Just a quick update on the proposal. We have decided to move
> >> quorum
> >> > > > > > reassignment to a separate KIP:
> >> > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> >> > > > > > .
> >> > > > > > The way this ties into cluster bootstrapping is complicated,
> so
> >> we
> >> > > felt
> >> > > > > we
> >> > > > > > needed a bit more time for validation. That leaves the core of
> >> this
> >> > > > > > proposal as quorum-based replication. If there are no further
> >> > > comments,
> >> > > > > we
> >> > > > > > will plan to start a vote later this week.
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Jason
> >> > > > > >
> >> > > > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <
> >> wangguoz@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > > @Jun Rao <ju...@gmail.com>
> >> > > > > > >
> >> > > > > > > Regarding your comment about log compaction. After some
> >> > deep-diving
> >> > > > > into
> >> > > > > > > this we've decided to propose a new snapshot-based log
> >> cleaning
> >> > > > > mechanism
> >> > > > > > > which would be used to replace the current compaction
> >> mechanism
> >> > for
> >> > > > > this
> >> > > > > > > meta log. A new KIP will be proposed specifically for this
> >> idea.
> >> > > > > > >
> >> > > > > > > All,
> >> > > > > > >
> >> > > > > > > I've updated the KIP wiki a bit updating one config "
> >> > > > > > > election.jitter.max.ms"
> >> > > > > > > to "election.backoff.max.ms" to make it more clear about
> the
> >> > > usage:
> >> > > > > the
> >> > > > > > > configured value will be the upper bound of the binary
> >> > exponential
> >> > > > > > backoff
> >> > > > > > > time after a failed election, before starting a new one.
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > Guozhang
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> >> > > > > reluctanthero104@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > >
> >> > > > > > > > Thanks for the suggestions Guozhang.
> >> > > > > > > >
> >> > > > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
> >> > > wangguoz@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Hello Boyang,
> >> > > > > > > > >
> >> > > > > > > > > Thanks for the updated information. A few questions
> here:
> >> > > > > > > > >
> >> > > > > > > > > 1) Should the quorum-file also update to support
> >> multi-raft?
> >> > > > > > > > >
> >> > > > > > > > > I'm neutral about this, as we don't know yet how the
> >> > multi-raft
> >> > > > > > modules
> >> > > > > > > > would behave. If
> >> > > > > > > > we have different threads operating different raft groups,
> >> > > > > > consolidating
> >> > > > > > > > the `checkpoint` files seems
> >> > > > > > > > not reasonable. We could always add `multi-quorum-file`
> >> later
> >> > if
> >> > > > > > > possible.
> >> > > > > > > >
> >> > > > > > > > 2) In the previous proposal, there's fields in the
> >> > > > FetchQuorumRecords
> >> > > > > > > like
> >> > > > > > > > > latestDirtyOffset, is that dropped intentionally?
> >> > > > > > > > >
> >> > > > > > > > > I dropped the latestDirtyOffset since it is associated
> >> with
> >> > the
> >> > > > log
> >> > > > > > > > compaction discussion. This is beyond this KIP scope and
> we
> >> > could
> >> > > > > > > > potentially get a separate KIP to talk about it.
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > > 3) I think we also need to elaborate a bit more details
> >> > > regarding
> >> > > > > > when
> >> > > > > > > to
> >> > > > > > > > > send metadata request and discover-brokers; currently we
> >> only
> >> > > > > > discussed
> >> > > > > > > > > during bootstrap how these requests would be sent. I
> think
> >> > the
> >> > > > > > > following
> >> > > > > > > > > scenarios would also need these requests
> >> > > > > > > > >
> >> > > > > > > > > 3.a) As long as a broker does not know the current
> quorum
> >> > > > > (including
> >> > > > > > > the
> >> > > > > > > > > leader and the voters), it should continue periodically
> >> ask
> >> > > other
> >> > > > > > > brokers
> >> > > > > > > > > via "metadata.
> >> > > > > > > > > 3.b) As long as a broker does not know all the current
> >> quorum
> >> > > > > voter's
> >> > > > > > > > > connections, it should continue periodically ask other
> >> > brokers
> >> > > > via
> >> > > > > > > > > "discover-brokers".
> >> > > > > > > > > 3.c) When the leader's fetch timeout elapsed, it should
> >> send
> >> > > > > metadata
> >> > > > > > > > > request.
> >> > > > > > > > >
> >> > > > > > > > > Make sense, will add to the KIP.
> >> > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > Guozhang
> >> > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> >> > > > > > > reluctanthero104@gmail.com>
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Hey all,
> >> > > > > > > > > >
> >> > > > > > > > > > follow-up on the previous email, we made some more
> >> updates:
> >> > > > > > > > > >
> >> > > > > > > > > > 1. The Alter/DescribeQuorum RPCs are also
> re-structured
> >> to
> >> > > use
> >> > > > > > > > > multi-raft.
> >> > > > > > > > > >
> >> > > > > > > > > > 2. We add observer status into the
> >> DescribeQuorumResponse
> >> > as
> >> > > we
> >> > > > > see
> >> > > > > > > it
> >> > > > > > > > > is a
> >> > > > > > > > > > low hanging fruit which is very useful for user
> >> debugging
> >> > and
> >> > > > > > > > > reassignment.
> >> > > > > > > > > >
> >> > > > > > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers
> >> RPC,
> >> > > > which
> >> > > > > > is
> >> > > > > > > > > purely
> >> > > > > > > > > > in charge of discovering broker connections in a
> gossip
> >> > > manner.
> >> > > > > The
> >> > > > > > > > > quorum
> >> > > > > > > > > > leader discovery is piggy-back on the Metadata RPC for
> >> the
> >> > > > topic
> >> > > > > > > > > partition
> >> > > > > > > > > > leader, which in our case is the single metadata
> >> partition
> >> > > for
> >> > > > > the
> >> > > > > > > > > version
> >> > > > > > > > > > one.
> >> > > > > > > > > >
> >> > > > > > > > > > Let me know if you have any questions.
> >> > > > > > > > > >
> >> > > > > > > > > > Boyang
> >> > > > > > > > > >
> >> > > > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> >> > > > > > > > reluctanthero104@gmail.com>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > Hey all,
> >> > > > > > > > > > >
> >> > > > > > > > > > > Thanks for the great discussions so far. I'm posting
> >> some
> >> > > KIP
> >> > > > > > > updates
> >> > > > > > > > > > from
> >> > > > > > > > > > > our working group discussion:
> >> > > > > > > > > > >
> >> > > > > > > > > > > 1. We will be changing the core RPCs from
> single-raft
> >> API
> >> > > to
> >> > > > > > > > > multi-raft.
> >> > > > > > > > > > > This means all protocols will be "batch" in the
> first
> >> > > > version,
> >> > > > > > but
> >> > > > > > > > the
> >> > > > > > > > > > KIP
> >> > > > > > > > > > > itself only illustrates the design for a single
> >> metadata
> >> > > > topic
> >> > > > > > > > > partition.
> >> > > > > > > > > > > The reason is to "keep the door open" for future
> >> > extensions
> >> > > > of
> >> > > > > > this
> >> > > > > > > > > piece
> >> > > > > > > > > > > of module such as a sharded controller or general
> >> quorum
> >> > > > based
> >> > > > > > > topic
> >> > > > > > > > > > > replication, beyond the current Kafka replication
> >> > protocol.
> >> > > > > > > > > > >
> >> > > > > > > > > > > 2. We will piggy-back on the current Kafka Fetch API
> >> > > instead
> >> > > > of
> >> > > > > > > > > inventing
> >> > > > > > > > > > > a new FetchQuorumRecords RPC. The motivation is
> about
> >> the
> >> > > > same
> >> > > > > as
> >> > > > > > > #1
> >> > > > > > > > as
> >> > > > > > > > > > > well as making the integration work easier, instead
> of
> >> > > > letting
> >> > > > > > two
> >> > > > > > > > > > similar
> >> > > > > > > > > > > RPCs diverge.
> >> > > > > > > > > > >
> >> > > > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of only
> >> > sending
> >> > > > the
> >> > > > > > > > request
> >> > > > > > > > > to
> >> > > > > > > > > > > the most caught-up voter, we shall broadcast the
> >> > > information
> >> > > > to
> >> > > > > > all
> >> > > > > > > > > > voters,
> >> > > > > > > > > > > with a sorted voter list in descending order of
> their
> >> > > > > > corresponding
> >> > > > > > > > > > > replicated offset. In this way, the top voter will
> >> > become a
> >> > > > > > > candidate
> >> > > > > > > > > > > immediately, while the other voters shall wait for
> an
> >> > > > > exponential
> >> > > > > > > > > > back-off
> >> > > > > > > > > > > to trigger elections, which helps ensure the top
> voter
> >> > gets
> >> > > > > > > elected,
> >> > > > > > > > > and
> >> > > > > > > > > > > the election eventually happens when the top voter
> is
> >> not
> >> > > > > > > responsive.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Please see the updated KIP and post any questions or
> >> > > concerns
> >> > > > > on
> >> > > > > > > the
> >> > > > > > > > > > > mailing thread.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Boyang
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <
> >> jun@confluent.io
> >> > >
> >> > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > >> Hi, Guozhang and Jason,
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> Thanks for the reply. A couple of more replies.
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> 102. Still not sure about this. How is the
> tombstone
> >> > issue
> >> > > > > > > addressed
> >> > > > > > > > > in
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> non-voter and the observer.  They can die at any
> >> point
> >> > and
> >> > > > > > restart
> >> > > > > > > > at
> >> > > > > > > > > an
> >> > > > > > > > > > >> arbitrary later time, and the advancing of the
> >> > firstDirty
> >> > > > > offset
> >> > > > > > > and
> >> > > > > > > > > the
> >> > > > > > > > > > >> removal of the tombstone can happen independently.
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> 106. I agree that it would be less confusing if we
> >> used
> >> > > > > "epoch"
> >> > > > > > > > > instead
> >> > > > > > > > > > of
> >> > > > > > > > > > >> "leader epoch" consistently.
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> Jun
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> >> > > > > > wangguoz@gmail.com>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> > Thanks Jun. Further replies are in-lined.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
> >> > > jun@confluent.io
> >> > > > >
> >> > > > > > > wrote:
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > > Hi, Guozhang,
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > Thanks for the reply. A few more replies
> inlined
> >> > > below.
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> >> > > > > > > > wangguoz@gmail.com>
> >> > > > > > > > > > >> wrote:
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > > Hello Jun,
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Thanks for your comments! I'm replying inline
> >> > below:
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> >> > > > > jun@confluent.io
> >> > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > > 101. Bootstrapping related issues.
> >> > > > > > > > > > >> > > > > 101.1 Currently, we support auto broker id
> >> > > > generation.
> >> > > > > > Is
> >> > > > > > > > this
> >> > > > > > > > > > >> > > supported
> >> > > > > > > > > > >> > > > > for bootstrap brokers?
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > The vote ids would just be the broker ids.
> >> > > > > > > "bootstrap.servers"
> >> > > > > > > > > > >> would be
> >> > > > > > > > > > >> > > > similar to what client configs have today,
> >> where
> >> > > > > > > > "quorum.voters"
> >> > > > > > > > > > >> would
> >> > > > > > > > > > >> > be
> >> > > > > > > > > > >> > > > pre-defined config values.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > My question was on the auto generated broker
> id.
> >> > > > > Currently,
> >> > > > > > > the
> >> > > > > > > > > > broker
> >> > > > > > > > > > >> > can
> >> > > > > > > > > > >> > > choose to have its broker Id auto generated.
> The
> >> > > > > generation
> >> > > > > > is
> >> > > > > > > > > done
> >> > > > > > > > > > >> > through
> >> > > > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's
> not
> >> > clear
> >> > > > how
> >> > > > > > the
> >> > > > > > > > > > broker
> >> > > > > > > > > > >> id
> >> > > > > > > > > > >> > is
> >> > > > > > > > > > >> > > auto generated. "quorum.voters" also can't be
> set
> >> > > > > statically
> >> > > > > > > if
> >> > > > > > > > > > broker
> >> > > > > > > > > > >> > ids
> >> > > > > > > > > > >> > > are auto generated.
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > Jason has explained some ideas that we've
> >> discussed
> >> > so
> >> > > > > far,
> >> > > > > > > the
> >> > > > > > > > > > >> reason we
> >> > > > > > > > > > >> > intentional did not include them so far is that
> we
> >> > feel
> >> > > it
> >> > > > > is
> >> > > > > > > > > out-side
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500
> we
> >> > > should
> >> > > > > > > > definitely
> >> > > > > > > > > > >> > address them though.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > On the high-level, our belief is that "joining a
> >> > quorum"
> >> > > > and
> >> > > > > > > > > "joining
> >> > > > > > > > > > >> (or
> >> > > > > > > > > > >> > more specifically, registering brokers in) the
> >> > cluster"
> >> > > > > would
> >> > > > > > be
> >> > > > > > > > > > >> > de-coupled a bit, where the former should be
> >> completed
> >> > > > > before
> >> > > > > > we
> >> > > > > > > > do
> >> > > > > > > > > > the
> >> > > > > > > > > > >> > latter. More specifically, assuming the quorum is
> >> > > already
> >> > > > up
> >> > > > > > and
> >> > > > > > > > > > >> running,
> >> > > > > > > > > > >> > after the newly started broker found the leader
> of
> >> the
> >> > > > > quorum
> >> > > > > > it
> >> > > > > > > > can
> >> > > > > > > > > > >> send a
> >> > > > > > > > > > >> > specific RegisterBroker request including its
> >> > listener /
> >> > > > > > > protocol
> >> > > > > > > > /
> >> > > > > > > > > > etc,
> >> > > > > > > > > > >> > and upon handling it the leader can send back the
> >> > > uniquely
> >> > > > > > > > generated
> >> > > > > > > > > > >> broker
> >> > > > > > > > > > >> > id to the new broker, while also executing the
> >> > > > > > "startNewBroker"
> >> > > > > > > > > > >> callback as
> >> > > > > > > > > > >> > the controller.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > > > 102. Log compaction. One weak spot of log
> >> > > compaction
> >> > > > > is
> >> > > > > > > for
> >> > > > > > > > > the
> >> > > > > > > > > > >> > > consumer
> >> > > > > > > > > > >> > > > to
> >> > > > > > > > > > >> > > > > deal with deletes. When a key is deleted,
> >> it's
> >> > > > > retained
> >> > > > > > > as a
> >> > > > > > > > > > >> > tombstone
> >> > > > > > > > > > >> > > > > first and then physically removed. If a
> >> client
> >> > > > misses
> >> > > > > > the
> >> > > > > > > > > > >> tombstone
> >> > > > > > > > > > >> > > > > (because it's physically removed), it may
> >> not be
> >> > > > able
> >> > > > > to
> >> > > > > > > > > update
> >> > > > > > > > > > >> its
> >> > > > > > > > > > >> > > > > metadata properly. The way we solve this in
> >> > Kafka
> >> > > is
> >> > > > > > based
> >> > > > > > > > on
> >> > > > > > > > > a
> >> > > > > > > > > > >> > > > > configuration (
> >> log.cleaner.delete.retention.ms)
> >> > > and
> >> > > > > we
> >> > > > > > > > > expect a
> >> > > > > > > > > > >> > > consumer
> >> > > > > > > > > > >> > > > > having seen an old key to finish reading
> the
> >> > > > deletion
> >> > > > > > > > > tombstone
> >> > > > > > > > > > >> > within
> >> > > > > > > > > > >> > > > that
> >> > > > > > > > > > >> > > > > time. There is no strong guarantee for that
> >> > since
> >> > > a
> >> > > > > > broker
> >> > > > > > > > > could
> >> > > > > > > > > > >> be
> >> > > > > > > > > > >> > > down
> >> > > > > > > > > > >> > > > > for a long time. It would be better if we
> can
> >> > > have a
> >> > > > > > more
> >> > > > > > > > > > reliable
> >> > > > > > > > > > >> > way
> >> > > > > > > > > > >> > > of
> >> > > > > > > > > > >> > > > > dealing with deletes.
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > We propose to capture this in the
> >> > "FirstDirtyOffset"
> >> > > > > field
> >> > > > > > > of
> >> > > > > > > > > the
> >> > > > > > > > > > >> > quorum
> >> > > > > > > > > > >> > > > record fetch response: the offset is the
> >> maximum
> >> > > > offset
> >> > > > > > that
> >> > > > > > > > log
> >> > > > > > > > > > >> > > compaction
> >> > > > > > > > > > >> > > > has reached up to. If the follower has
> fetched
> >> > > beyond
> >> > > > > this
> >> > > > > > > > > offset
> >> > > > > > > > > > it
> >> > > > > > > > > > >> > > means
> >> > > > > > > > > > >> > > > itself is safe hence it has seen all records
> >> up to
> >> > > > that
> >> > > > > > > > offset.
> >> > > > > > > > > On
> >> > > > > > > > > > >> > > getting
> >> > > > > > > > > > >> > > > the response, the follower can then decide if
> >> its
> >> > > end
> >> > > > > > offset
> >> > > > > > > > > > >> actually
> >> > > > > > > > > > >> > > below
> >> > > > > > > > > > >> > > > that dirty offset (and hence may miss some
> >> > > > tombstones).
> >> > > > > If
> >> > > > > > > > > that's
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > case:
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > 1) Naively, it could re-bootstrap metadata
> log
> >> > from
> >> > > > the
> >> > > > > > very
> >> > > > > > > > > > >> beginning
> >> > > > > > > > > > >> > to
> >> > > > > > > > > > >> > > > catch up.
> >> > > > > > > > > > >> > > > 2) During that time, it would refrain itself
> >> from
> >> > > > > > answering
> >> > > > > > > > > > >> > > MetadataRequest
> >> > > > > > > > > > >> > > > from any clients.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > I am not sure that the "FirstDirtyOffset" field
> >> > fully
> >> > > > > > > addresses
> >> > > > > > > > > the
> >> > > > > > > > > > >> > issue.
> >> > > > > > > > > > >> > > Currently, the deletion tombstone is not
> removed
> >> > > > > immediately
> >> > > > > > > > > after a
> >> > > > > > > > > > >> > round
> >> > > > > > > > > > >> > > of cleaning. It's removed after a delay in a
> >> > > subsequent
> >> > > > > > round
> >> > > > > > > of
> >> > > > > > > > > > >> > cleaning.
> >> > > > > > > > > > >> > > Consider an example where a key insertion is at
> >> > offset
> >> > > > 200
> >> > > > > > > and a
> >> > > > > > > > > > >> deletion
> >> > > > > > > > > > >> > > tombstone of the key is at 400. Initially,
> >> > > > > FirstDirtyOffset
> >> > > > > > is
> >> > > > > > > > at
> >> > > > > > > > > > >> 300. A
> >> > > > > > > > > > >> > > follower/observer fetches from offset 0  and
> >> fetches
> >> > > the
> >> > > > > key
> >> > > > > > > at
> >> > > > > > > > > > offset
> >> > > > > > > > > > >> > 200.
> >> > > > > > > > > > >> > > A few rounds of cleaning happen.
> >> FirstDirtyOffset is
> >> > > at
> >> > > > > 500
> >> > > > > > > and
> >> > > > > > > > > the
> >> > > > > > > > > > >> > > tombstone at 400 is physically removed. The
> >> > > > > > follower/observer
> >> > > > > > > > > > >> continues
> >> > > > > > > > > > >> > the
> >> > > > > > > > > > >> > > fetch, but misses offset 400. It catches all
> the
> >> way
> >> > > to
> >> > > > > > > > > > >> FirstDirtyOffset
> >> > > > > > > > > > >> > > and declares its metadata as ready. However,
> its
> >> > > > metadata
> >> > > > > > > could
> >> > > > > > > > be
> >> > > > > > > > > > >> stale
> >> > > > > > > > > > >> > > since it actually misses the deletion of the
> key.
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > Yeah good question, I should have put more
> >> details
> >> > in
> >> > > my
> >> > > > > > > > > explanation
> >> > > > > > > > > > >> :)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > The idea is that we will adjust the log
> compaction
> >> for
> >> > > > this
> >> > > > > > raft
> >> > > > > > > > > based
> >> > > > > > > > > > >> > metadata log: before more details to be
> explained,
> >> > since
> >> > > > we
> >> > > > > > have
> >> > > > > > > > two
> >> > > > > > > > > > >> types
> >> > > > > > > > > > >> > of "watermarks" here, whereas in Kafka the
> >> watermark
> >> > > > > indicates
> >> > > > > > > > where
> >> > > > > > > > > > >> every
> >> > > > > > > > > > >> > replica have replicated up to and in Raft the
> >> > watermark
> >> > > > > > > indicates
> >> > > > > > > > > > where
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > majority of replicas (here only indicating voters
> >> of
> >> > the
> >> > > > > > quorum,
> >> > > > > > > > not
> >> > > > > > > > > > >> > counting observers) have replicated up to, let's
> >> call
> >> > > them
> >> > > > > > Kafka
> >> > > > > > > > > > >> watermark
> >> > > > > > > > > > >> > and Raft watermark. For this special log, we
> would
> >> > > > maintain
> >> > > > > > both
> >> > > > > > > > > > >> > watermarks.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > When log compacting on the leader, we would only
> >> > compact
> >> > > > up
> >> > > > > to
> >> > > > > > > the
> >> > > > > > > > > > Kafka
> >> > > > > > > > > > >> > watermark, i.e. if there is at least one voter
> who
> >> > have
> >> > > > not
> >> > > > > > > > > replicated
> >> > > > > > > > > > >> an
> >> > > > > > > > > > >> > entry, it would not be compacted. The
> >> "dirty-offset"
> >> > is
> >> > > > the
> >> > > > > > > offset
> >> > > > > > > > > > that
> >> > > > > > > > > > >> > we've compacted up to and is communicated to
> other
> >> > > voters,
> >> > > > > and
> >> > > > > > > the
> >> > > > > > > > > > other
> >> > > > > > > > > > >> > voters would also compact up to this value ---
> i.e.
> >> > the
> >> > > > > > > difference
> >> > > > > > > > > > here
> >> > > > > > > > > > >> is
> >> > > > > > > > > > >> > that instead of letting each replica doing log
> >> > > compaction
> >> > > > > > > > > > independently,
> >> > > > > > > > > > >> > we'll have the leader to decide upon which offset
> >> to
> >> > > > compact
> >> > > > > > to,
> >> > > > > > > > and
> >> > > > > > > > > > >> > propagate this value to others to follow, in a
> more
> >> > > > > > coordinated
> >> > > > > > > > > > manner.
> >> > > > > > > > > > >> > Also note when there are new voters joining the
> >> quorum
> >> > > who
> >> > > > > has
> >> > > > > > > not
> >> > > > > > > > > > >> > replicated up to the dirty-offset, of because of
> >> other
> >> > > > > issues
> >> > > > > > > they
> >> > > > > > > > > > >> > truncated their logs to below the dirty-offset,
> >> they'd
> >> > > > have
> >> > > > > to
> >> > > > > > > > > > >> re-bootstrap
> >> > > > > > > > > > >> > from the beginning, and during this period of
> time
> >> the
> >> > > > > leader
> >> > > > > > > > > learned
> >> > > > > > > > > > >> about
> >> > > > > > > > > > >> > this lagging voter would not advance the
> watermark
> >> > (also
> >> > > > it
> >> > > > > > > would
> >> > > > > > > > > not
> >> > > > > > > > > > >> > decrement it), and hence not compacting either,
> >> until
> >> > > the
> >> > > > > > > voter(s)
> >> > > > > > > > > has
> >> > > > > > > > > > >> > caught up to that dirty-offset.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > So back to your example above, before the
> bootstrap
> >> > > voter
> >> > > > > gets
> >> > > > > > > to
> >> > > > > > > > > 300
> >> > > > > > > > > > no
> >> > > > > > > > > > >> > log compaction would happen on the leader; and
> >> until
> >> > > later
> >> > > > > > when
> >> > > > > > > > the
> >> > > > > > > > > > >> voter
> >> > > > > > > > > > >> > have got to beyond 400 and hence replicated that
> >> > > > tombstone,
> >> > > > > > the
> >> > > > > > > > log
> >> > > > > > > > > > >> > compaction would possibly get to that tombstone
> and
> >> > > remove
> >> > > > > it.
> >> > > > > > > Say
> >> > > > > > > > > > >> later it
> >> > > > > > > > > > >> > the leader's log compaction reaches 500, it can
> >> send
> >> > > this
> >> > > > > back
> >> > > > > > > to
> >> > > > > > > > > the
> >> > > > > > > > > > >> voter
> >> > > > > > > > > > >> > who can then also compact locally up to 500.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > > > > 105. Quorum State: In addition to VotedId,
> >> do we
> >> > > > need
> >> > > > > > the
> >> > > > > > > > > epoch
> >> > > > > > > > > > >> > > > > corresponding to VotedId? Over time, the
> same
> >> > > broker
> >> > > > > Id
> >> > > > > > > > could
> >> > > > > > > > > be
> >> > > > > > > > > > >> > voted
> >> > > > > > > > > > >> > > in
> >> > > > > > > > > > >> > > > > different generations with different epoch.
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > Hmm, this is a good point. Originally I think
> >> the
> >> > > > > > > > "LeaderEpoch"
> >> > > > > > > > > > >> field
> >> > > > > > > > > > >> > in
> >> > > > > > > > > > >> > > > that file is corresponding to the "latest
> known
> >> > > leader
> >> > > > > > > epoch",
> >> > > > > > > > > not
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > > "current leader epoch". For example, if the
> >> > current
> >> > > > > epoch
> >> > > > > > is
> >> > > > > > > > N,
> >> > > > > > > > > > and
> >> > > > > > > > > > >> > then
> >> > > > > > > > > > >> > > a
> >> > > > > > > > > > >> > > > vote-request with epoch N+1 is received and
> the
> >> > > voter
> >> > > > > > > granted
> >> > > > > > > > > the
> >> > > > > > > > > > >> vote
> >> > > > > > > > > > >> > > for
> >> > > > > > > > > > >> > > > it, then it means for this voter it knows the
> >> > > "latest
> >> > > > > > epoch"
> >> > > > > > > > is
> >> > > > > > > > > N
> >> > > > > > > > > > +
> >> > > > > > > > > > >> 1
> >> > > > > > > > > > >> > > > although it is unknown if that sending
> >> candidate
> >> > > will
> >> > > > > > indeed
> >> > > > > > > > > > become
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > new
> >> > > > > > > > > > >> > > > leader (which would only be notified via
> >> > > begin-quorum
> >> > > > > > > > request).
> >> > > > > > > > > > >> > However,
> >> > > > > > > > > > >> > > > when persisting the quorum state, we would
> >> encode
> >> > > > > > > leader-epoch
> >> > > > > > > > > to
> >> > > > > > > > > > >> N+1,
> >> > > > > > > > > > >> > > > while the leaderId to be the older leader.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > But now thinking about this a bit more, I
> feel
> >> we
> >> > > > should
> >> > > > > > use
> >> > > > > > > > two
> >> > > > > > > > > > >> > separate
> >> > > > > > > > > > >> > > > epochs, one for the "lates known" and one for
> >> the
> >> > > > > > "current"
> >> > > > > > > to
> >> > > > > > > > > > pair
> >> > > > > > > > > > >> > with
> >> > > > > > > > > > >> > > > the leaderId. I will update the wiki page.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > Hmm, it's kind of weird to bump up the leader
> >> epoch
> >> > > > before
> >> > > > > > the
> >> > > > > > > > new
> >> > > > > > > > > > >> leader
> >> > > > > > > > > > >> > > is actually elected, right.
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> >> > > > > > FetchQuorumRecords
> >> > > > > > > > API
> >> > > > > > > > > to
> >> > > > > > > > > > >> > > indicate
> >> > > > > > > > > > >> > > > > that the follower has fetched from an
> invalid
> >> > > offset
> >> > > > > and
> >> > > > > > > > > should
> >> > > > > > > > > > >> > > truncate
> >> > > > > > > > > > >> > > > to
> >> > > > > > > > > > >> > > > > the offset/epoch indicated in the
> response."
> >> > > > Observers
> >> > > > > > > can't
> >> > > > > > > > > > >> truncate
> >> > > > > > > > > > >> > > > their
> >> > > > > > > > > > >> > > > > logs. What should they do with
> >> > > OFFSET_OUT_OF_RANGE?
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > I'm not sure if I understand your question?
> >> > > Observers
> >> > > > > > should
> >> > > > > > > > > still
> >> > > > > > > > > > >> be
> >> > > > > > > > > > >> > > able
> >> > > > > > > > > > >> > > > to truncate their logs as well.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > Hmm, I thought only the quorum nodes have local
> >> logs
> >> > > and
> >> > > > > > > > observers
> >> > > > > > > > > > >> don't?
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > > 107. "The leader will continue sending
> >> > > > BeginQuorumEpoch
> >> > > > > to
> >> > > > > > > > each
> >> > > > > > > > > > >> known
> >> > > > > > > > > > >> > > > voter
> >> > > > > > > > > > >> > > > > until it has received its endorsement." If
> a
> >> > voter
> >> > > > is
> >> > > > > > down
> >> > > > > > > > > for a
> >> > > > > > > > > > >> long
> >> > > > > > > > > > >> > > > time,
> >> > > > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add
> >> > unnecessary
> >> > > > > > > overhead.
> >> > > > > > > > > > >> > Similarly,
> >> > > > > > > > > > >> > > > if a
> >> > > > > > > > > > >> > > > > follower stops sending FetchQuorumRecords,
> >> does
> >> > > the
> >> > > > > > leader
> >> > > > > > > > > keep
> >> > > > > > > > > > >> > sending
> >> > > > > > > > > > >> > > > > BeginQuorumEpoch?
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good
> >> point.
> >> > > The
> >> > > > > > > > > > >> > begin-quorum-epoch
> >> > > > > > > > > > >> > > > request is for voters to quickly get the new
> >> > leader
> >> > > > > > > > information;
> >> > > > > > > > > > >> > however
> >> > > > > > > > > > >> > > > even if they do not get them they can still
> >> > > eventually
> >> > > > > > learn
> >> > > > > > > > > about
> >> > > > > > > > > > >> that
> >> > > > > > > > > > >> > > > from others via gossiping FindQuorum. I think
> >> we
> >> > can
> >> > > > > > adjust
> >> > > > > > > > the
> >> > > > > > > > > > >> logic
> >> > > > > > > > > > >> > to
> >> > > > > > > > > > >> > > > e.g. exponential back-off or with a limited
> >> > > > num.retries.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Regarding FetchQuorumRecords: if the follower
> >> > sends
> >> > > > > > > > > > >> FetchQuorumRecords
> >> > > > > > > > > > >> > > > already, it means that follower already knows
> >> that
> >> > > the
> >> > > > > > > broker
> >> > > > > > > > is
> >> > > > > > > > > > the
> >> > > > > > > > > > >> > > > leader, and hence we can stop retrying
> >> > > > BeginQuorumEpoch;
> >> > > > > > > > however
> >> > > > > > > > > > it
> >> > > > > > > > > > >> is
> >> > > > > > > > > > >> > > > possible that after a follower sends
> >> > > > FetchQuorumRecords
> >> > > > > > > > already,
> >> > > > > > > > > > >> > suddenly
> >> > > > > > > > > > >> > > > it stops send it (possibly because it learned
> >> > about
> >> > > a
> >> > > > > > higher
> >> > > > > > > > > epoch
> >> > > > > > > > > > >> > > leader),
> >> > > > > > > > > > >> > > > and hence this broker may be a "zombie"
> leader
> >> and
> >> > > we
> >> > > > > > > propose
> >> > > > > > > > to
> >> > > > > > > > > > use
> >> > > > > > > > > > >> > the
> >> > > > > > > > > > >> > > > fetch.timeout to let the leader to try to
> >> verify
> >> > if
> >> > > it
> >> > > > > has
> >> > > > > > > > > already
> >> > > > > > > > > > >> been
> >> > > > > > > > > > >> > > > stale.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > It just seems that we should handle these two
> >> cases
> >> > > in a
> >> > > > > > > > > consistent
> >> > > > > > > > > > >> way?
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > Yes I agree, on the leader's side, the
> >> > > > FetchQuorumRecords
> >> > > > > > > from a
> >> > > > > > > > > > >> follower
> >> > > > > > > > > > >> > could mean that we no longer needs to send
> >> > > > BeginQuorumEpoch
> >> > > > > > > > anymore
> >> > > > > > > > > > ---
> >> > > > > > > > > > >> and
> >> > > > > > > > > > >> > it is already part of our current implementations
> >> in
> >> > > > > > > > > > >> >
> >> > > https://github.com/confluentinc/kafka/commits/kafka-raft
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > > Thanks,
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > Jun
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > Jun
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang
> >> Wang <
> >> > > > > > > > > > wangguoz@gmail.com
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > > > wrote:
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > > Hello Leonard,
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > Thanks for your comments, I'm relying in
> >> line
> >> > > > below:
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang
> >> (Leonard)
> >> > > Ge
> >> > > > <
> >> > > > > > > > > > >> > wge@confluent.io>
> >> > > > > > > > > > >> > > > > > wrote:
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > > Hi Kafka developers,
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > > It's great to see this proposal and it
> >> took
> >> > me
> >> > > > > some
> >> > > > > > > time
> >> > > > > > > > > to
> >> > > > > > > > > > >> > finish
> >> > > > > > > > > > >> > > > > > reading
> >> > > > > > > > > > >> > > > > > > it.
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > > And I have the following questions
> about
> >> the
> >> > > > > > Proposal:
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > >    - How do we plan to test this design
> >> to
> >> > > > ensure
> >> > > > > > its
> >> > > > > > > > > > >> > correctness?
> >> > > > > > > > > > >> > > Or
> >> > > > > > > > > > >> > > > > > more
> >> > > > > > > > > > >> > > > > > >    broadly, how do we ensure that our
> new
> >> > > ‘pull’
> >> > > > > > based
> >> > > > > > > > > model
> >> > > > > > > > > > >> is
> >> > > > > > > > > > >> > > > > > functional
> >> > > > > > > > > > >> > > > > > > and
> >> > > > > > > > > > >> > > > > > >    correct given that it is different
> >> from
> >> > the
> >> > > > > > > original
> >> > > > > > > > > RAFT
> >> > > > > > > > > > >> > > > > > implementation
> >> > > > > > > > > > >> > > > > > >    which has formal proof of
> correctness?
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > We have two planned verifications on the
> >> > > > correctness
> >> > > > > > and
> >> > > > > > > > > > >> liveness
> >> > > > > > > > > > >> > of
> >> > > > > > > > > > >> > > > the
> >> > > > > > > > > > >> > > > > > design. One is via model verification
> >> (TLA+)
> >> > > > > > > > > > >> > > > > >
> >> > > > https://github.com/guozhangwang/kafka-specification
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > Another is via the concurrent simulation
> >> tests
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >>
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > >    - Have we considered any sensible
> >> defaults
> >> > > for
> >> > > > > the
> >> > > > > > > > > > >> > configuration,
> >> > > > > > > > > > >> > > > i.e.
> >> > > > > > > > > > >> > > > > > >    all the election timeout, fetch time
> >> out,
> >> > > > etc.?
> >> > > > > > Or
> >> > > > > > > we
> >> > > > > > > > > > want
> >> > > > > > > > > > >> to
> >> > > > > > > > > > >> > > > leave
> >> > > > > > > > > > >> > > > > > > this to
> >> > > > > > > > > > >> > > > > > >    a later stage when we do the
> >> performance
> >> > > > > testing,
> >> > > > > > > > etc.
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > This is a good question, the reason we
> did
> >> not
> >> > > set
> >> > > > > any
> >> > > > > > > > > default
> >> > > > > > > > > > >> > values
> >> > > > > > > > > > >> > > > for
> >> > > > > > > > > > >> > > > > > the timeout configurations is that we
> >> think it
> >> > > may
> >> > > > > > take
> >> > > > > > > > some
> >> > > > > > > > > > >> > > > benchmarking
> >> > > > > > > > > > >> > > > > > experiments to get these defaults right.
> >> Some
> >> > > > > > high-level
> >> > > > > > > > > > >> principles
> >> > > > > > > > > > >> > > to
> >> > > > > > > > > > >> > > > > > consider: 1) the fetch.timeout should be
> >> > around
> >> > > > the
> >> > > > > > same
> >> > > > > > > > > scale
> >> > > > > > > > > > >> with
> >> > > > > > > > > > >> > > zk
> >> > > > > > > > > > >> > > > > > session timeout, which is now 18 seconds
> by
> >> > > > default
> >> > > > > --
> >> > > > > > > in
> >> > > > > > > > > > >> practice
> >> > > > > > > > > > >> > > > we've
> >> > > > > > > > > > >> > > > > > seen unstable networks having more than
> 10
> >> > secs
> >> > > of
> >> > > > > > > > transient
> >> > > > > > > > > > >> > > > > connectivity,
> >> > > > > > > > > > >> > > > > > 2) the election.timeout, however, should
> be
> >> > > > smaller
> >> > > > > > than
> >> > > > > > > > the
> >> > > > > > > > > > >> fetch
> >> > > > > > > > > > >> > > > > timeout
> >> > > > > > > > > > >> > > > > > as is also suggested as a practical
> >> > optimization
> >> > > > in
> >> > > > > > > > > > literature:
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > >
> >> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > Some more discussions can be found here:
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > >
> >> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > >    - Have we considered piggybacking
> >> > > > > > > `BeginQuorumEpoch`
> >> > > > > > > > > with
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > `
> >> > > > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be
> >> missing
> >> > > > > something
> >> > > > > > > > > obvious
> >> > > > > > > > > > >> but
> >> > > > > > > > > > >> > I
> >> > > > > > > > > > >> > > am
> >> > > > > > > > > > >> > > > > > just
> >> > > > > > > > > > >> > > > > > >    wondering why don’t we just use the
> >> > > > > `FindQuorum`
> >> > > > > > > and
> >> > > > > > > > > > >> > > > > > > `FetchQuorumRecords`
> >> > > > > > > > > > >> > > > > > >    APIs and remove the
> `BeginQuorumEpoch`
> >> > API?
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent
> from
> >> > > leader
> >> > > > > ->
> >> > > > > > > > other
> >> > > > > > > > > > >> voter
> >> > > > > > > > > > >> > > > > > followers, while FindQuorum / Fetch are
> >> sent
> >> > > from
> >> > > > > > > follower
> >> > > > > > > > > to
> >> > > > > > > > > > >> > leader.
> >> > > > > > > > > > >> > > > > > Arguably one can eventually realize the
> new
> >> > > leader
> >> > > > > and
> >> > > > > > > > epoch
> >> > > > > > > > > > via
> >> > > > > > > > > > >> > > > > gossiping
> >> > > > > > > > > > >> > > > > > FindQuorum, but that could in practice
> >> > require a
> >> > > > > long
> >> > > > > > > > delay.
> >> > > > > > > > > > >> > Having a
> >> > > > > > > > > > >> > > > > > leader -> other voters request helps the
> >> new
> >> > > > leader
> >> > > > > > > epoch
> >> > > > > > > > to
> >> > > > > > > > > > be
> >> > > > > > > > > > >> > > > > propagated
> >> > > > > > > > > > >> > > > > > faster under a pull model.
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > >    - And about the `FetchQuorumRecords`
> >> > > response
> >> > > > > > > schema,
> >> > > > > > > > > in
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > > > `Records`
> >> > > > > > > > > > >> > > > > > >    field of the response, is it just
> one
> >> > > record
> >> > > > or
> >> > > > > > all
> >> > > > > > > > the
> >> > > > > > > > > > >> > records
> >> > > > > > > > > > >> > > > > > starting
> >> > > > > > > > > > >> > > > > > >    from the FetchOffset? It seems a lot
> >> more
> >> > > > > > efficient
> >> > > > > > > > if
> >> > > > > > > > > we
> >> > > > > > > > > > >> sent
> >> > > > > > > > > > >> > > all
> >> > > > > > > > > > >> > > > > the
> >> > > > > > > > > > >> > > > > > >    records during the bootstrapping of
> >> the
> >> > > > > brokers.
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > Yes the fetching is batched: FetchOffset
> is
> >> > just
> >> > > > the
> >> > > > > > > > > starting
> >> > > > > > > > > > >> > offset
> >> > > > > > > > > > >> > > of
> >> > > > > > > > > > >> > > > > the
> >> > > > > > > > > > >> > > > > > batch of records.
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > >    - Regarding the disruptive broker
> >> issues,
> >> > > > does
> >> > > > > > our
> >> > > > > > > > pull
> >> > > > > > > > > > >> based
> >> > > > > > > > > > >> > > > model
> >> > > > > > > > > > >> > > > > > >    suffer from it? If so, have we
> >> considered
> >> > > the
> >> > > > > > > > Pre-Vote
> >> > > > > > > > > > >> stage?
> >> > > > > > > > > > >> > If
> >> > > > > > > > > > >> > > > > not,
> >> > > > > > > > > > >> > > > > > > why?
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > The disruptive broker is stated in the
> >> > original
> >> > > > Raft
> >> > > > > > > paper
> >> > > > > > > > > > >> which is
> >> > > > > > > > > > >> > > the
> >> > > > > > > > > > >> > > > > > result of the push model design. Our
> >> analysis
> >> > > > showed
> >> > > > > > > that
> >> > > > > > > > > with
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > pull
> >> > > > > > > > > > >> > > > > > model it is no longer an issue.
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > > Thanks a lot for putting this up, and I
> >> hope
> >> > > > that
> >> > > > > my
> >> > > > > > > > > > questions
> >> > > > > > > > > > >> > can
> >> > > > > > > > > > >> > > be
> >> > > > > > > > > > >> > > > > of
> >> > > > > > > > > > >> > > > > > > some value to make this KIP better.
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > > Hope to hear from you soon!
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > > Best wishes,
> >> > > > > > > > > > >> > > > > > > Leonard
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin
> >> > McCabe <
> >> > > > > > > > > > >> cmccabe@apache.org
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > > > wrote:
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > > > Hi Jason,
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > It's amazing to see this coming
> >> together
> >> > :)
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > I haven't had a chance to read in
> >> detail,
> >> > > but
> >> > > > I
> >> > > > > > read
> >> > > > > > > > the
> >> > > > > > > > > > >> > outline
> >> > > > > > > > > > >> > > > and
> >> > > > > > > > > > >> > > > > a
> >> > > > > > > > > > >> > > > > > > few
> >> > > > > > > > > > >> > > > > > > > things jumped out at me.
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > First, for every epoch that is 32
> bits
> >> > > rather
> >> > > > > than
> >> > > > > > > > 64, I
> >> > > > > > > > > > >> sort
> >> > > > > > > > > > >> > of
> >> > > > > > > > > > >> > > > > wonder
> >> > > > > > > > > > >> > > > > > > if
> >> > > > > > > > > > >> > > > > > > > that's a good long-term choice.  I
> keep
> >> > > > reading
> >> > > > > > > about
> >> > > > > > > > > > stuff
> >> > > > > > > > > > >> > like
> >> > > > > > > > > > >> > > > > this:
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> >> > > > > > > > .
> >> > > > > > > > > > >> > > Obviously,
> >> > > > > > > > > > >> > > > > > that
> >> > > > > > > > > > >> > > > > > > > JIRA is about zxid, which increments
> >> much
> >> > > > faster
> >> > > > > > > than
> >> > > > > > > > we
> >> > > > > > > > > > >> expect
> >> > > > > > > > > > >> > > > these
> >> > > > > > > > > > >> > > > > > > > leader epochs to, but it would still
> be
> >> > good
> >> > > > to
> >> > > > > > see
> >> > > > > > > > some
> >> > > > > > > > > > >> rough
> >> > > > > > > > > > >> > > > > > > calculations
> >> > > > > > > > > > >> > > > > > > > about how long 32 bits (or really, 31
> >> > bits)
> >> > > > will
> >> > > > > > > last
> >> > > > > > > > us
> >> > > > > > > > > > in
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > > cases
> >> > > > > > > > > > >> > > > > > > where
> >> > > > > > > > > > >> > > > > > > > we're using it, and what the space
> >> savings
> >> > > > we're
> >> > > > > > > > getting
> >> > > > > > > > > > >> really
> >> > > > > > > > > > >> > > is.
> >> > > > > > > > > > >> > > > > It
> >> > > > > > > > > > >> > > > > > > > seems like in most cases the tradeoff
> >> may
> >> > > not
> >> > > > be
> >> > > > > > > worth
> >> > > > > > > > > it?
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > Another thing I've been thinking
> about
> >> is
> >> > > how
> >> > > > we
> >> > > > > > do
> >> > > > > > > > > > >> > > > bootstrapping.  I
> >> > > > > > > > > > >> > > > > > > > would prefer to be in a world where
> >> > > > formatting a
> >> > > > > > new
> >> > > > > > > > > Kafka
> >> > > > > > > > > > >> node
> >> > > > > > > > > > >> > > > was a
> >> > > > > > > > > > >> > > > > > > first
> >> > > > > > > > > > >> > > > > > > > class operation explicitly initiated
> by
> >> > the
> >> > > > > admin,
> >> > > > > > > > > rather
> >> > > > > > > > > > >> than
> >> > > > > > > > > > >> > > > > > something
> >> > > > > > > > > > >> > > > > > > > that happened implicitly when you
> >> started
> >> > up
> >> > > > the
> >> > > > > > > > broker
> >> > > > > > > > > > and
> >> > > > > > > > > > >> > > things
> >> > > > > > > > > > >> > > > > > > "looked
> >> > > > > > > > > > >> > > > > > > > blank."
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > The first problem is that things can
> >> "look
> >> > > > > blank"
> >> > > > > > > > > > >> accidentally
> >> > > > > > > > > > >> > if
> >> > > > > > > > > > >> > > > the
> >> > > > > > > > > > >> > > > > > > > storage system is having a bad day.
> >> > Clearly
> >> > > > in
> >> > > > > > the
> >> > > > > > > > > > non-Raft
> >> > > > > > > > > > >> > > world,
> >> > > > > > > > > > >> > > > > > this
> >> > > > > > > > > > >> > > > > > > > leads to data loss if the broker that
> >> is
> >> > > > > > (re)started
> >> > > > > > > > > this
> >> > > > > > > > > > >> way
> >> > > > > > > > > > >> > was
> >> > > > > > > > > > >> > > > the
> >> > > > > > > > > > >> > > > > > > > leader for some partitions.
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > The second problem is that we have a
> >> bit
> >> > of
> >> > > a
> >> > > > > > > chicken
> >> > > > > > > > > and
> >> > > > > > > > > > >> egg
> >> > > > > > > > > > >> > > > problem
> >> > > > > > > > > > >> > > > > > > with
> >> > > > > > > > > > >> > > > > > > > certain configuration keys.  For
> >> example,
> >> > > > maybe
> >> > > > > > you
> >> > > > > > > > want
> >> > > > > > > > > > to
> >> > > > > > > > > > >> > > > configure
> >> > > > > > > > > > >> > > > > > > some
> >> > > > > > > > > > >> > > > > > > > connection security settings in your
> >> > > cluster,
> >> > > > > but
> >> > > > > > > you
> >> > > > > > > > > > don't
> >> > > > > > > > > > >> > want
> >> > > > > > > > > > >> > > > them
> >> > > > > > > > > > >> > > > > > to
> >> > > > > > > > > > >> > > > > > > > ever be stored in a plaintext config
> >> file.
> >> > > > (For
> >> > > > > > > > > example,
> >> > > > > > > > > > >> SCRAM
> >> > > > > > > > > > >> > > > > > > passwords,
> >> > > > > > > > > > >> > > > > > > > etc.)  You could use a broker API to
> >> set
> >> > the
> >> > > > > > > > > > configuration,
> >> > > > > > > > > > >> but
> >> > > > > > > > > > >> > > > that
> >> > > > > > > > > > >> > > > > > > brings
> >> > > > > > > > > > >> > > > > > > > up the chicken and egg problem.  The
> >> > broker
> >> > > > > needs
> >> > > > > > to
> >> > > > > > > > be
> >> > > > > > > > > > >> > > configured
> >> > > > > > > > > > >> > > > to
> >> > > > > > > > > > >> > > > > > > know
> >> > > > > > > > > > >> > > > > > > > how to talk to you, but you need to
> >> > > configure
> >> > > > it
> >> > > > > > > > before
> >> > > > > > > > > > you
> >> > > > > > > > > > >> can
> >> > > > > > > > > > >> > > > talk
> >> > > > > > > > > > >> > > > > to
> >> > > > > > > > > > >> > > > > > > > it.  Using an external secret manager
> >> like
> >> > > > Vault
> >> > > > > > is
> >> > > > > > > > one
> >> > > > > > > > > > way
> >> > > > > > > > > > >> to
> >> > > > > > > > > > >> > > > solve
> >> > > > > > > > > > >> > > > > > > this,
> >> > > > > > > > > > >> > > > > > > > but not everyone uses an external
> >> secret
> >> > > > > manager.
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > quorum.voters seems like a similar
> >> > > > configuration
> >> > > > > > > key.
> >> > > > > > > > > In
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > > current
> >> > > > > > > > > > >> > > > > > > KIP,
> >> > > > > > > > > > >> > > > > > > > this is only read if there is no
> other
> >> > > > > > configuration
> >> > > > > > > > > > >> specifying
> >> > > > > > > > > > >> > > the
> >> > > > > > > > > > >> > > > > > > quorum
> >> > > > > > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs
> >> > command,
> >> > > we
> >> > > > > > > > wouldn't
> >> > > > > > > > > > need
> >> > > > > > > > > > >> > this
> >> > > > > > > > > > >> > > > key
> >> > > > > > > > > > >> > > > > > > > because we could assume that there
> was
> >> > > always
> >> > > > > > quorum
> >> > > > > > > > > > >> > information
> >> > > > > > > > > > >> > > > > stored
> >> > > > > > > > > > >> > > > > > > > locally.
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > best,
> >> > > > > > > > > > >> > > > > > > > Colin
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason
> >> > > > Gustafson
> >> > > > > > > wrote:
> >> > > > > > > > > > >> > > > > > > > > Hi All,
> >> > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > >> > > > > > > > > I'd like to start a discussion on
> >> > KIP-595:
> >> > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >>
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> >> > > > > > > > > > >> > > > > > > > .
> >> > > > > > > > > > >> > > > > > > > > This proposal specifies a Raft
> >> protocol
> >> > to
> >> > > > > > > > ultimately
> >> > > > > > > > > > >> replace
> >> > > > > > > > > > >> > > > > > Zookeeper
> >> > > > > > > > > > >> > > > > > > > > as
> >> > > > > > > > > > >> > > > > > > > > documented in KIP-500. Please take
> a
> >> > look
> >> > > > and
> >> > > > > > > share
> >> > > > > > > > > your
> >> > > > > > > > > > >> > > > thoughts.
> >> > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > >> > > > > > > > > A few minor notes to set the stage
> a
> >> > > little
> >> > > > > bit:
> >> > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > >> > > > > > > > > - This KIP does not specify the
> >> > structure
> >> > > of
> >> > > > > the
> >> > > > > > > > > > messages
> >> > > > > > > > > > >> > used
> >> > > > > > > > > > >> > > to
> >> > > > > > > > > > >> > > > > > > > represent
> >> > > > > > > > > > >> > > > > > > > > metadata in Kafka, nor does it
> >> specify
> >> > the
> >> > > > > > > internal
> >> > > > > > > > > API
> >> > > > > > > > > > >> that
> >> > > > > > > > > > >> > > will
> >> > > > > > > > > > >> > > > > be
> >> > > > > > > > > > >> > > > > > > used
> >> > > > > > > > > > >> > > > > > > > > by the controller. Expect these to
> >> come
> >> > in
> >> > > > > later
> >> > > > > > > > > > >> proposals.
> >> > > > > > > > > > >> > > Here
> >> > > > > > > > > > >> > > > we
> >> > > > > > > > > > >> > > > > > are
> >> > > > > > > > > > >> > > > > > > > > primarily concerned with the
> >> replication
> >> > > > > > protocol
> >> > > > > > > > and
> >> > > > > > > > > > >> basic
> >> > > > > > > > > > >> > > > > > operational
> >> > > > > > > > > > >> > > > > > > > > mechanics.
> >> > > > > > > > > > >> > > > > > > > > - We expect many details to change
> >> as we
> >> > > get
> >> > > > > > > closer
> >> > > > > > > > to
> >> > > > > > > > > > >> > > > integration
> >> > > > > > > > > > >> > > > > > with
> >> > > > > > > > > > >> > > > > > > > > the controller. Any changes we make
> >> will
> >> > > be
> >> > > > > made
> >> > > > > > > > > either
> >> > > > > > > > > > as
> >> > > > > > > > > > >> > > > > amendments
> >> > > > > > > > > > >> > > > > > > to
> >> > > > > > > > > > >> > > > > > > > > this KIP or, in the case of larger
> >> > > changes,
> >> > > > as
> >> > > > > > new
> >> > > > > > > > > > >> proposals.
> >> > > > > > > > > > >> > > > > > > > > - We have a prototype
> implementation
> >> > > which I
> >> > > > > > will
> >> > > > > > > > put
> >> > > > > > > > > > >> online
> >> > > > > > > > > > >> > > > within
> >> > > > > > > > > > >> > > > > > the
> >> > > > > > > > > > >> > > > > > > > > next week which may help in
> >> > understanding
> >> > > > some
> >> > > > > > > > > details.
> >> > > > > > > > > > It
> >> > > > > > > > > > >> > has
> >> > > > > > > > > > >> > > > > > > diverged a
> >> > > > > > > > > > >> > > > > > > > > little bit from our proposal, so I
> am
> >> > > > taking a
> >> > > > > > > > little
> >> > > > > > > > > > >> time to
> >> > > > > > > > > > >> > > > bring
> >> > > > > > > > > > >> > > > > > it
> >> > > > > > > > > > >> > > > > > > in
> >> > > > > > > > > > >> > > > > > > > > line. I'll post an update to this
> >> thread
> >> > > > when
> >> > > > > it
> >> > > > > > > is
> >> > > > > > > > > > >> available
> >> > > > > > > > > > >> > > for
> >> > > > > > > > > > >> > > > > > > review.
> >> > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > >> > > > > > > > > Finally, I want to mention that
> this
> >> > > > proposal
> >> > > > > > was
> >> > > > > > > > > > drafted
> >> > > > > > > > > > >> by
> >> > > > > > > > > > >> > > > > myself,
> >> > > > > > > > > > >> > > > > > > > Boyang
> >> > > > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> >> > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > >> > > > > > > > > Thanks,
> >> > > > > > > > > > >> > > > > > > > > Jason
> >> > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > > > --
> >> > > > > > > > > > >> > > > > > > Leonard Ge
> >> > > > > > > > > > >> > > > > > > Software Engineer Intern - Confluent
> >> > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > > > --
> >> > > > > > > > > > >> > > > > > -- Guozhang
> >> > > > > > > > > > >> > > > > >
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > --
> >> > > > > > > > > > >> > > > -- Guozhang
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > --
> >> > > > > > > > > > >> > -- Guozhang
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >>
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > --
> >> > > > > > > > > -- Guozhang
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > --
> >> > > > > > > -- Guozhang
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Unmesh Joshi <un...@gmail.com>.
Just checked etcd and zookeeper code, and both support leader to step down
as a follower to make sure there are no two leaders if the leader has been
disconnected from the majority of the followers
For etcd this is https://github.com/etcd-io/etcd/issues/3866
For Zookeeper its https://issues.apache.org/jira/browse/ZOOKEEPER-1699
I was just thinking if it would be difficult to implement in the Pull based
model, but I guess not. It is possibly the same way ISR list is managed
currently, if leader of the controller quorum loses majority of the
followers, it should step down and become follower, that way, telling
client in time that it was disconnected from the quorum, and not keep on
sending state metadata to clients.

Thanks,
Unmesh


On Mon, Jul 27, 2020 at 9:31 AM Unmesh Joshi <un...@gmail.com> wrote:

> >>Could you clarify on this question? Which part of the raft group doesn't
> >>know about leader dis-connection?
> The leader of the controller quorum is partitioned from the controller
> cluster, and a different leader is elected for the remaining controller
> cluster.
> I think there are two things here,
> 1.  The old leader will not know if it's disconnected from the rest of the
> controller quorum cluster unless it receives BeginQuorumEpoch from the new
> leader. So it will keep on serving stale metadata to the clients (Brokers,
> Producers and Consumers)
> 2. I assume, the Broker Leases will be managed on the controller quorum
> leader. This partitioned leader will keep on tracking broker leases it has,
> while the new leader of the quorum will also start managing broker leases.
> So while the quorum leader is partitioned, there will be two membership
> views of the kafka brokers managed on two leaders.
> Unless broker heartbeats are also replicated as part of the Raft log,
> there is no way to solve this?
> I know LogCabin implementation does replicate client heartbeats. I suspect
> that the same issue is there in Zookeeper, which does not replicate client
> Ping requests..
>
> Thanks,
> Unmesh
>
>
>
> On Mon, Jul 27, 2020 at 6:23 AM Boyang Chen <re...@gmail.com>
> wrote:
>
>> Thanks for the questions Unmesh!
>>
>> On Sun, Jul 26, 2020 at 6:18 AM Unmesh Joshi <un...@gmail.com>
>> wrote:
>>
>> > Hi,
>> >
>> > In the FetchRequest Handling, how to make sure we handle scenarios where
>> > the leader might have been disconnected from the cluster, but doesn't
>> know
>> > yet?
>> >
>> Could you clarify on this question? Which part of the raft group doesn't
>> know about leader
>> dis-connection?
>>
>>
>> > As discussed in the Raft Thesis section 6.4, the linearizable semantics
>> of
>> > read requests is implemented in LogCabin by sending heartbeat to
>> followers
>> > and waiting till the heartbeats are successful to make sure that the
>> leader
>> > is still the leader.
>> > I think for the controller quorum to make sure none of the consumers get
>> > stale data, it's important to have linearizable semantics? In the pull
>> > based model, the leader will need to wait for heartbeats from the
>> followers
>> > before returning each fetch request from the consumer then? Or do we
>> need
>> > to introduce some other request?
>> > (Zookeeper does not have linearizable semantics for read requests, but
>> as
>> > of now all the kafka interactions are through writes and watches).
>> >
>> > This is a very good question. For our v1 implementation we are not
>> aiming
>> to guarantee linearizable read, which
>> would be considered as a follow-up effort. Note that today in Kafka there
>> is no guarantee on the metadata freshness either,
>> so no regression is introduced.
>>
>>
>> > Thanks,
>> > Unmesh
>> >
>> > On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Hi, Jason,
>> > >
>> > > Thanks for the reply.
>> > >
>> > > 101. Sounds good. Regarding clusterId, I am not sure storing it in the
>> > > metadata log is enough. For example, the vote request includes
>> clusterId.
>> > > So, no one can vote until they know the clusterId. Also, it would be
>> > useful
>> > > to support the case when a voter completely loses its disk and needs
>> to
>> > > recover.
>> > >
>> > > 210. There is no longer a FindQuorum request. When a follower
>> restarts,
>> > how
>> > > does it discover the leader? Is that based on DescribeQuorum? It
>> would be
>> > > useful to document this.
>> > >
>> > > Jun
>> > >
>> > > On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <ja...@confluent.io>
>> > > wrote:
>> > >
>> > > > Hi Jun,
>> > > >
>> > > > Thanks for the questions.
>> > > >
>> > > > 101. I am treating some of the bootstrapping problems as out of the
>> > scope
>> > > > of this KIP. I am working on a separate proposal which addresses
>> > > > bootstrapping security credentials specifically. Here is a rough
>> sketch
>> > > of
>> > > > how I am seeing it:
>> > > >
>> > > > 1. Dynamic broker configurations including encrypted passwords will
>> be
>> > > > persisted in the metadata log and cached in the broker's
>> > > `meta.properties`
>> > > > file.
>> > > > 2. We will provide a tool which allows users to directly override
>> the
>> > > > values in `meta.properties` without requiring access to the quorum.
>> > This
>> > > > can be used to bootstrap the credentials of the voter set itself
>> before
>> > > the
>> > > > cluster has been started.
>> > > > 3. Some dynamic config changes will only be allowed when a broker is
>> > > > online. For example, changing a truststore password dynamically
>> would
>> > > > prevent that broker from being able to start if it were offline when
>> > the
>> > > > change was made.
>> > > > 4. I am still thinking a little bit about SCRAM credentials, but
>> most
>> > > > likely they will be handled with an approach similar to
>> > > `meta.properties`.
>> > > >
>> > > > 101.3 As for the question about `clusterId`, I think the way we
>> would
>> > do
>> > > > this is to have the first elected leader generate a UUID and write
>> it
>> > to
>> > > > the metadata log. Let me add some detail to the proposal about this.
>> > > >
>> > > > A few additional answers below:
>> > > >
>> > > > 203. Yes, that is correct.
>> > > >
>> > > > 204. That is a good question. What happens in this case is that all
>> > > voters
>> > > > advance their epoch to the one designated by the candidate even if
>> they
>> > > > reject its vote request. Assuming the candidate fails to be elected,
>> > the
>> > > > election will be retried until a leader emerges.
>> > > >
>> > > > 205. I had some discussion with Colin offline about this problem. I
>> > think
>> > > > the answer should be "yes," but it probably needs a little more
>> > thought.
>> > > > Handling JBOD failures is tricky. For an observer, we can replicate
>> the
>> > > > metadata log from scratch safely in a new log dir. But if the log
>> dir
>> > of
>> > > a
>> > > > voter fails, I do not think it is generally safe to start from an
>> empty
>> > > > state.
>> > > >
>> > > > 206. Yes, that is discussed in KIP-631 I believe.
>> > > >
>> > > > 207. Good suggestion. I will work on this.
>> > > >
>> > > >
>> > > > Thanks,
>> > > > Jason
>> > > >
>> > > >
>> > > >
>> > > >
>> > > >
>> > > > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:
>> > > >
>> > > > > Hi, Jason,
>> > > > >
>> > > > > Thanks for the updated KIP. Looks good overall. A few more
>> comments
>> > > > below.
>> > > > >
>> > > > > 101. I still don't see a section on bootstrapping related issues.
>> It
>> > > > would
>> > > > > be useful to document if/how the following is supported.
>> > > > > 101.1 Currently, we support auto broker id generation. Is this
>> > > supported
>> > > > > for bootstrap brokers?
>> > > > > 101.2 As Colin mentioned, sometimes we may need to load the
>> security
>> > > > > credentials to be broker before it can be connected to. Could you
>> > > > provide a
>> > > > > bit more detail on how this will work?
>> > > > > 101.3 Currently, we use ZK to generate clusterId on a new cluster.
>> > With
>> > > > > Raft, how does every broker generate the same clusterId in a
>> > > distributed
>> > > > > way?
>> > > > >
>> > > > > 200. It would be useful to document if the various special offsets
>> > (log
>> > > > > start offset, recovery point, HWM, etc) for the Raft log are
>> stored
>> > in
>> > > > the
>> > > > > same existing checkpoint files or not.
>> > > > > 200.1 Since the Raft log flushes every append, does that allow us
>> to
>> > > > > recover from a recovery point within the active segment or do we
>> > still
>> > > > need
>> > > > > to scan the full segment including the recovery point? The former
>> can
>> > > be
>> > > > > tricky since multiple records can fall into the same disk page
>> and a
>> > > > > subsequent flush may corrupt a page with previously flushed
>> records.
>> > > > >
>> > > > > 201. Configurations.
>> > > > > 201.1 How do the Raft brokers get security related configs for
>> inter
>> > > > broker
>> > > > > communication? Is that based on the existing
>> > > > > inter.broker.security.protocol?
>> > > > > 201.2 We have quorum.retry.backoff.max.ms and
>> > quorum.retry.backoff.ms,
>> > > > but
>> > > > > only quorum.election.backoff.max.ms. This seems a bit
>> inconsistent.
>> > > > >
>> > > > > 202. Metrics:
>> > > > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
>> > > OutboundQueueTimeMs:
>> > > > > Are those the same as existing totalTime, requestQueueTime,
>> > localTime,
>> > > > > responseQueueTime? Could we reuse the existing ones with the tag
>> > > > > request=[request-type]?
>> > > > > 202.2. Could you explain what InboundChannelSize and
>> > > OutboundChannelSize
>> > > > > are?
>> > > > > 202.3 ElectionLatencyMax/Avg: It seems that both should be
>> windowed?
>> > > > >
>> > > > > 203. Quorum State: I assume that LeaderId will be kept
>> consistently
>> > > with
>> > > > > LeaderEpoch. For example, if a follower transitions to candidate
>> and
>> > > > bumps
>> > > > > up LeaderEpoch, it will set leaderId to -1 and persist both in the
>> > > Quorum
>> > > > > state file. Is that correct?
>> > > > >
>> > > > > 204. I was thinking about a corner case when a Raft broker is
>> > > partitioned
>> > > > > off. This broker will then be in a continuous loop of bumping up
>> the
>> > > > leader
>> > > > > epoch, but failing to get enough votes. When the partitioning is
>> > > removed,
>> > > > > this broker's high leader epoch will force a leader election. I
>> > assume
>> > > > > other Raft brokers can immediately advance their leader epoch
>> passing
>> > > the
>> > > > > already bumped epoch such that leader election won't be delayed.
>> Is
>> > > that
>> > > > > right?
>> > > > >
>> > > > > 205. In a JBOD setting, could we use the existing tool to move the
>> > Raft
>> > > > log
>> > > > > from one disk to another?
>> > > > >
>> > > > > 206. The KIP doesn't mention the local metadata store derived from
>> > the
>> > > > Raft
>> > > > > log. Will that be covered in a separate KIP?
>> > > > >
>> > > > > 207. Since this is a critical component. Could we add a section on
>> > the
>> > > > > testing plan for correctness?
>> > > > >
>> > > > > 208. Performance. Do we plan to do group commit (e.g. buffer
>> pending
>> > > > > appends during a flush and then flush all accumulated pending
>> records
>> > > > > together in the next flush) for better throughput?
>> > > > >
>> > > > > 209. "the leader can actually defer fsync until it knows
>> > "quorum.size -
>> > > > 1"
>> > > > > has get to a certain entry offset." Why is that "quorum.size - 1"
>> > > instead
>> > > > > of the majority of the quorum?
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <
>> jason@confluent.io>
>> > > > > wrote:
>> > > > >
>> > > > > > Hi All,
>> > > > > >
>> > > > > > Just a quick update on the proposal. We have decided to move
>> quorum
>> > > > > > reassignment to a separate KIP:
>> > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
>> > > > > > .
>> > > > > > The way this ties into cluster bootstrapping is complicated, so
>> we
>> > > felt
>> > > > > we
>> > > > > > needed a bit more time for validation. That leaves the core of
>> this
>> > > > > > proposal as quorum-based replication. If there are no further
>> > > comments,
>> > > > > we
>> > > > > > will plan to start a vote later this week.
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Jason
>> > > > > >
>> > > > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <
>> wangguoz@gmail.com
>> > >
>> > > > > wrote:
>> > > > > >
>> > > > > > > @Jun Rao <ju...@gmail.com>
>> > > > > > >
>> > > > > > > Regarding your comment about log compaction. After some
>> > deep-diving
>> > > > > into
>> > > > > > > this we've decided to propose a new snapshot-based log
>> cleaning
>> > > > > mechanism
>> > > > > > > which would be used to replace the current compaction
>> mechanism
>> > for
>> > > > > this
>> > > > > > > meta log. A new KIP will be proposed specifically for this
>> idea.
>> > > > > > >
>> > > > > > > All,
>> > > > > > >
>> > > > > > > I've updated the KIP wiki a bit updating one config "
>> > > > > > > election.jitter.max.ms"
>> > > > > > > to "election.backoff.max.ms" to make it more clear about the
>> > > usage:
>> > > > > the
>> > > > > > > configured value will be the upper bound of the binary
>> > exponential
>> > > > > > backoff
>> > > > > > > time after a failed election, before starting a new one.
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > Guozhang
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
>> > > > > reluctanthero104@gmail.com>
>> > > > > > > wrote:
>> > > > > > >
>> > > > > > > > Thanks for the suggestions Guozhang.
>> > > > > > > >
>> > > > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
>> > > wangguoz@gmail.com>
>> > > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Hello Boyang,
>> > > > > > > > >
>> > > > > > > > > Thanks for the updated information. A few questions here:
>> > > > > > > > >
>> > > > > > > > > 1) Should the quorum-file also update to support
>> multi-raft?
>> > > > > > > > >
>> > > > > > > > > I'm neutral about this, as we don't know yet how the
>> > multi-raft
>> > > > > > modules
>> > > > > > > > would behave. If
>> > > > > > > > we have different threads operating different raft groups,
>> > > > > > consolidating
>> > > > > > > > the `checkpoint` files seems
>> > > > > > > > not reasonable. We could always add `multi-quorum-file`
>> later
>> > if
>> > > > > > > possible.
>> > > > > > > >
>> > > > > > > > 2) In the previous proposal, there's fields in the
>> > > > FetchQuorumRecords
>> > > > > > > like
>> > > > > > > > > latestDirtyOffset, is that dropped intentionally?
>> > > > > > > > >
>> > > > > > > > > I dropped the latestDirtyOffset since it is associated
>> with
>> > the
>> > > > log
>> > > > > > > > compaction discussion. This is beyond this KIP scope and we
>> > could
>> > > > > > > > potentially get a separate KIP to talk about it.
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > > 3) I think we also need to elaborate a bit more details
>> > > regarding
>> > > > > > when
>> > > > > > > to
>> > > > > > > > > send metadata request and discover-brokers; currently we
>> only
>> > > > > > discussed
>> > > > > > > > > during bootstrap how these requests would be sent. I think
>> > the
>> > > > > > > following
>> > > > > > > > > scenarios would also need these requests
>> > > > > > > > >
>> > > > > > > > > 3.a) As long as a broker does not know the current quorum
>> > > > > (including
>> > > > > > > the
>> > > > > > > > > leader and the voters), it should continue periodically
>> ask
>> > > other
>> > > > > > > brokers
>> > > > > > > > > via "metadata.
>> > > > > > > > > 3.b) As long as a broker does not know all the current
>> quorum
>> > > > > voter's
>> > > > > > > > > connections, it should continue periodically ask other
>> > brokers
>> > > > via
>> > > > > > > > > "discover-brokers".
>> > > > > > > > > 3.c) When the leader's fetch timeout elapsed, it should
>> send
>> > > > > metadata
>> > > > > > > > > request.
>> > > > > > > > >
>> > > > > > > > > Make sense, will add to the KIP.
>> > > > > > > >
>> > > > > > > > >
>> > > > > > > > > Guozhang
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
>> > > > > > > reluctanthero104@gmail.com>
>> > > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > Hey all,
>> > > > > > > > > >
>> > > > > > > > > > follow-up on the previous email, we made some more
>> updates:
>> > > > > > > > > >
>> > > > > > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured
>> to
>> > > use
>> > > > > > > > > multi-raft.
>> > > > > > > > > >
>> > > > > > > > > > 2. We add observer status into the
>> DescribeQuorumResponse
>> > as
>> > > we
>> > > > > see
>> > > > > > > it
>> > > > > > > > > is a
>> > > > > > > > > > low hanging fruit which is very useful for user
>> debugging
>> > and
>> > > > > > > > > reassignment.
>> > > > > > > > > >
>> > > > > > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers
>> RPC,
>> > > > which
>> > > > > > is
>> > > > > > > > > purely
>> > > > > > > > > > in charge of discovering broker connections in a gossip
>> > > manner.
>> > > > > The
>> > > > > > > > > quorum
>> > > > > > > > > > leader discovery is piggy-back on the Metadata RPC for
>> the
>> > > > topic
>> > > > > > > > > partition
>> > > > > > > > > > leader, which in our case is the single metadata
>> partition
>> > > for
>> > > > > the
>> > > > > > > > > version
>> > > > > > > > > > one.
>> > > > > > > > > >
>> > > > > > > > > > Let me know if you have any questions.
>> > > > > > > > > >
>> > > > > > > > > > Boyang
>> > > > > > > > > >
>> > > > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
>> > > > > > > > reluctanthero104@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > Hey all,
>> > > > > > > > > > >
>> > > > > > > > > > > Thanks for the great discussions so far. I'm posting
>> some
>> > > KIP
>> > > > > > > updates
>> > > > > > > > > > from
>> > > > > > > > > > > our working group discussion:
>> > > > > > > > > > >
>> > > > > > > > > > > 1. We will be changing the core RPCs from single-raft
>> API
>> > > to
>> > > > > > > > > multi-raft.
>> > > > > > > > > > > This means all protocols will be "batch" in the first
>> > > > version,
>> > > > > > but
>> > > > > > > > the
>> > > > > > > > > > KIP
>> > > > > > > > > > > itself only illustrates the design for a single
>> metadata
>> > > > topic
>> > > > > > > > > partition.
>> > > > > > > > > > > The reason is to "keep the door open" for future
>> > extensions
>> > > > of
>> > > > > > this
>> > > > > > > > > piece
>> > > > > > > > > > > of module such as a sharded controller or general
>> quorum
>> > > > based
>> > > > > > > topic
>> > > > > > > > > > > replication, beyond the current Kafka replication
>> > protocol.
>> > > > > > > > > > >
>> > > > > > > > > > > 2. We will piggy-back on the current Kafka Fetch API
>> > > instead
>> > > > of
>> > > > > > > > > inventing
>> > > > > > > > > > > a new FetchQuorumRecords RPC. The motivation is about
>> the
>> > > > same
>> > > > > as
>> > > > > > > #1
>> > > > > > > > as
>> > > > > > > > > > > well as making the integration work easier, instead of
>> > > > letting
>> > > > > > two
>> > > > > > > > > > similar
>> > > > > > > > > > > RPCs diverge.
>> > > > > > > > > > >
>> > > > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of only
>> > sending
>> > > > the
>> > > > > > > > request
>> > > > > > > > > to
>> > > > > > > > > > > the most caught-up voter, we shall broadcast the
>> > > information
>> > > > to
>> > > > > > all
>> > > > > > > > > > voters,
>> > > > > > > > > > > with a sorted voter list in descending order of their
>> > > > > > corresponding
>> > > > > > > > > > > replicated offset. In this way, the top voter will
>> > become a
>> > > > > > > candidate
>> > > > > > > > > > > immediately, while the other voters shall wait for an
>> > > > > exponential
>> > > > > > > > > > back-off
>> > > > > > > > > > > to trigger elections, which helps ensure the top voter
>> > gets
>> > > > > > > elected,
>> > > > > > > > > and
>> > > > > > > > > > > the election eventually happens when the top voter is
>> not
>> > > > > > > responsive.
>> > > > > > > > > > >
>> > > > > > > > > > > Please see the updated KIP and post any questions or
>> > > concerns
>> > > > > on
>> > > > > > > the
>> > > > > > > > > > > mailing thread.
>> > > > > > > > > > >
>> > > > > > > > > > > Boyang
>> > > > > > > > > > >
>> > > > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <
>> jun@confluent.io
>> > >
>> > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > >> Hi, Guozhang and Jason,
>> > > > > > > > > > >>
>> > > > > > > > > > >> Thanks for the reply. A couple of more replies.
>> > > > > > > > > > >>
>> > > > > > > > > > >> 102. Still not sure about this. How is the tombstone
>> > issue
>> > > > > > > addressed
>> > > > > > > > > in
>> > > > > > > > > > >> the
>> > > > > > > > > > >> non-voter and the observer.  They can die at any
>> point
>> > and
>> > > > > > restart
>> > > > > > > > at
>> > > > > > > > > an
>> > > > > > > > > > >> arbitrary later time, and the advancing of the
>> > firstDirty
>> > > > > offset
>> > > > > > > and
>> > > > > > > > > the
>> > > > > > > > > > >> removal of the tombstone can happen independently.
>> > > > > > > > > > >>
>> > > > > > > > > > >> 106. I agree that it would be less confusing if we
>> used
>> > > > > "epoch"
>> > > > > > > > > instead
>> > > > > > > > > > of
>> > > > > > > > > > >> "leader epoch" consistently.
>> > > > > > > > > > >>
>> > > > > > > > > > >> Jun
>> > > > > > > > > > >>
>> > > > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
>> > > > > > wangguoz@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > >>
>> > > > > > > > > > >> > Thanks Jun. Further replies are in-lined.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
>> > > jun@confluent.io
>> > > > >
>> > > > > > > wrote:
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > > Hi, Guozhang,
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > Thanks for the reply. A few more replies inlined
>> > > below.
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
>> > > > > > > > wangguoz@gmail.com>
>> > > > > > > > > > >> wrote:
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > > Hello Jun,
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Thanks for your comments! I'm replying inline
>> > below:
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
>> > > > > jun@confluent.io
>> > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > > 101. Bootstrapping related issues.
>> > > > > > > > > > >> > > > > 101.1 Currently, we support auto broker id
>> > > > generation.
>> > > > > > Is
>> > > > > > > > this
>> > > > > > > > > > >> > > supported
>> > > > > > > > > > >> > > > > for bootstrap brokers?
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > The vote ids would just be the broker ids.
>> > > > > > > "bootstrap.servers"
>> > > > > > > > > > >> would be
>> > > > > > > > > > >> > > > similar to what client configs have today,
>> where
>> > > > > > > > "quorum.voters"
>> > > > > > > > > > >> would
>> > > > > > > > > > >> > be
>> > > > > > > > > > >> > > > pre-defined config values.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > My question was on the auto generated broker id.
>> > > > > Currently,
>> > > > > > > the
>> > > > > > > > > > broker
>> > > > > > > > > > >> > can
>> > > > > > > > > > >> > > choose to have its broker Id auto generated. The
>> > > > > generation
>> > > > > > is
>> > > > > > > > > done
>> > > > > > > > > > >> > through
>> > > > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not
>> > clear
>> > > > how
>> > > > > > the
>> > > > > > > > > > broker
>> > > > > > > > > > >> id
>> > > > > > > > > > >> > is
>> > > > > > > > > > >> > > auto generated. "quorum.voters" also can't be set
>> > > > > statically
>> > > > > > > if
>> > > > > > > > > > broker
>> > > > > > > > > > >> > ids
>> > > > > > > > > > >> > > are auto generated.
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > Jason has explained some ideas that we've
>> discussed
>> > so
>> > > > > far,
>> > > > > > > the
>> > > > > > > > > > >> reason we
>> > > > > > > > > > >> > intentional did not include them so far is that we
>> > feel
>> > > it
>> > > > > is
>> > > > > > > > > out-side
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we
>> > > should
>> > > > > > > > definitely
>> > > > > > > > > > >> > address them though.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > On the high-level, our belief is that "joining a
>> > quorum"
>> > > > and
>> > > > > > > > > "joining
>> > > > > > > > > > >> (or
>> > > > > > > > > > >> > more specifically, registering brokers in) the
>> > cluster"
>> > > > > would
>> > > > > > be
>> > > > > > > > > > >> > de-coupled a bit, where the former should be
>> completed
>> > > > > before
>> > > > > > we
>> > > > > > > > do
>> > > > > > > > > > the
>> > > > > > > > > > >> > latter. More specifically, assuming the quorum is
>> > > already
>> > > > up
>> > > > > > and
>> > > > > > > > > > >> running,
>> > > > > > > > > > >> > after the newly started broker found the leader of
>> the
>> > > > > quorum
>> > > > > > it
>> > > > > > > > can
>> > > > > > > > > > >> send a
>> > > > > > > > > > >> > specific RegisterBroker request including its
>> > listener /
>> > > > > > > protocol
>> > > > > > > > /
>> > > > > > > > > > etc,
>> > > > > > > > > > >> > and upon handling it the leader can send back the
>> > > uniquely
>> > > > > > > > generated
>> > > > > > > > > > >> broker
>> > > > > > > > > > >> > id to the new broker, while also executing the
>> > > > > > "startNewBroker"
>> > > > > > > > > > >> callback as
>> > > > > > > > > > >> > the controller.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > > > 102. Log compaction. One weak spot of log
>> > > compaction
>> > > > > is
>> > > > > > > for
>> > > > > > > > > the
>> > > > > > > > > > >> > > consumer
>> > > > > > > > > > >> > > > to
>> > > > > > > > > > >> > > > > deal with deletes. When a key is deleted,
>> it's
>> > > > > retained
>> > > > > > > as a
>> > > > > > > > > > >> > tombstone
>> > > > > > > > > > >> > > > > first and then physically removed. If a
>> client
>> > > > misses
>> > > > > > the
>> > > > > > > > > > >> tombstone
>> > > > > > > > > > >> > > > > (because it's physically removed), it may
>> not be
>> > > > able
>> > > > > to
>> > > > > > > > > update
>> > > > > > > > > > >> its
>> > > > > > > > > > >> > > > > metadata properly. The way we solve this in
>> > Kafka
>> > > is
>> > > > > > based
>> > > > > > > > on
>> > > > > > > > > a
>> > > > > > > > > > >> > > > > configuration (
>> log.cleaner.delete.retention.ms)
>> > > and
>> > > > > we
>> > > > > > > > > expect a
>> > > > > > > > > > >> > > consumer
>> > > > > > > > > > >> > > > > having seen an old key to finish reading the
>> > > > deletion
>> > > > > > > > > tombstone
>> > > > > > > > > > >> > within
>> > > > > > > > > > >> > > > that
>> > > > > > > > > > >> > > > > time. There is no strong guarantee for that
>> > since
>> > > a
>> > > > > > broker
>> > > > > > > > > could
>> > > > > > > > > > >> be
>> > > > > > > > > > >> > > down
>> > > > > > > > > > >> > > > > for a long time. It would be better if we can
>> > > have a
>> > > > > > more
>> > > > > > > > > > reliable
>> > > > > > > > > > >> > way
>> > > > > > > > > > >> > > of
>> > > > > > > > > > >> > > > > dealing with deletes.
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > We propose to capture this in the
>> > "FirstDirtyOffset"
>> > > > > field
>> > > > > > > of
>> > > > > > > > > the
>> > > > > > > > > > >> > quorum
>> > > > > > > > > > >> > > > record fetch response: the offset is the
>> maximum
>> > > > offset
>> > > > > > that
>> > > > > > > > log
>> > > > > > > > > > >> > > compaction
>> > > > > > > > > > >> > > > has reached up to. If the follower has fetched
>> > > beyond
>> > > > > this
>> > > > > > > > > offset
>> > > > > > > > > > it
>> > > > > > > > > > >> > > means
>> > > > > > > > > > >> > > > itself is safe hence it has seen all records
>> up to
>> > > > that
>> > > > > > > > offset.
>> > > > > > > > > On
>> > > > > > > > > > >> > > getting
>> > > > > > > > > > >> > > > the response, the follower can then decide if
>> its
>> > > end
>> > > > > > offset
>> > > > > > > > > > >> actually
>> > > > > > > > > > >> > > below
>> > > > > > > > > > >> > > > that dirty offset (and hence may miss some
>> > > > tombstones).
>> > > > > If
>> > > > > > > > > that's
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > case:
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log
>> > from
>> > > > the
>> > > > > > very
>> > > > > > > > > > >> beginning
>> > > > > > > > > > >> > to
>> > > > > > > > > > >> > > > catch up.
>> > > > > > > > > > >> > > > 2) During that time, it would refrain itself
>> from
>> > > > > > answering
>> > > > > > > > > > >> > > MetadataRequest
>> > > > > > > > > > >> > > > from any clients.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > I am not sure that the "FirstDirtyOffset" field
>> > fully
>> > > > > > > addresses
>> > > > > > > > > the
>> > > > > > > > > > >> > issue.
>> > > > > > > > > > >> > > Currently, the deletion tombstone is not removed
>> > > > > immediately
>> > > > > > > > > after a
>> > > > > > > > > > >> > round
>> > > > > > > > > > >> > > of cleaning. It's removed after a delay in a
>> > > subsequent
>> > > > > > round
>> > > > > > > of
>> > > > > > > > > > >> > cleaning.
>> > > > > > > > > > >> > > Consider an example where a key insertion is at
>> > offset
>> > > > 200
>> > > > > > > and a
>> > > > > > > > > > >> deletion
>> > > > > > > > > > >> > > tombstone of the key is at 400. Initially,
>> > > > > FirstDirtyOffset
>> > > > > > is
>> > > > > > > > at
>> > > > > > > > > > >> 300. A
>> > > > > > > > > > >> > > follower/observer fetches from offset 0  and
>> fetches
>> > > the
>> > > > > key
>> > > > > > > at
>> > > > > > > > > > offset
>> > > > > > > > > > >> > 200.
>> > > > > > > > > > >> > > A few rounds of cleaning happen.
>> FirstDirtyOffset is
>> > > at
>> > > > > 500
>> > > > > > > and
>> > > > > > > > > the
>> > > > > > > > > > >> > > tombstone at 400 is physically removed. The
>> > > > > > follower/observer
>> > > > > > > > > > >> continues
>> > > > > > > > > > >> > the
>> > > > > > > > > > >> > > fetch, but misses offset 400. It catches all the
>> way
>> > > to
>> > > > > > > > > > >> FirstDirtyOffset
>> > > > > > > > > > >> > > and declares its metadata as ready. However, its
>> > > > metadata
>> > > > > > > could
>> > > > > > > > be
>> > > > > > > > > > >> stale
>> > > > > > > > > > >> > > since it actually misses the deletion of the key.
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > Yeah good question, I should have put more
>> details
>> > in
>> > > my
>> > > > > > > > > explanation
>> > > > > > > > > > >> :)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > The idea is that we will adjust the log compaction
>> for
>> > > > this
>> > > > > > raft
>> > > > > > > > > based
>> > > > > > > > > > >> > metadata log: before more details to be explained,
>> > since
>> > > > we
>> > > > > > have
>> > > > > > > > two
>> > > > > > > > > > >> types
>> > > > > > > > > > >> > of "watermarks" here, whereas in Kafka the
>> watermark
>> > > > > indicates
>> > > > > > > > where
>> > > > > > > > > > >> every
>> > > > > > > > > > >> > replica have replicated up to and in Raft the
>> > watermark
>> > > > > > > indicates
>> > > > > > > > > > where
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > majority of replicas (here only indicating voters
>> of
>> > the
>> > > > > > quorum,
>> > > > > > > > not
>> > > > > > > > > > >> > counting observers) have replicated up to, let's
>> call
>> > > them
>> > > > > > Kafka
>> > > > > > > > > > >> watermark
>> > > > > > > > > > >> > and Raft watermark. For this special log, we would
>> > > > maintain
>> > > > > > both
>> > > > > > > > > > >> > watermarks.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > When log compacting on the leader, we would only
>> > compact
>> > > > up
>> > > > > to
>> > > > > > > the
>> > > > > > > > > > Kafka
>> > > > > > > > > > >> > watermark, i.e. if there is at least one voter who
>> > have
>> > > > not
>> > > > > > > > > replicated
>> > > > > > > > > > >> an
>> > > > > > > > > > >> > entry, it would not be compacted. The
>> "dirty-offset"
>> > is
>> > > > the
>> > > > > > > offset
>> > > > > > > > > > that
>> > > > > > > > > > >> > we've compacted up to and is communicated to other
>> > > voters,
>> > > > > and
>> > > > > > > the
>> > > > > > > > > > other
>> > > > > > > > > > >> > voters would also compact up to this value --- i.e.
>> > the
>> > > > > > > difference
>> > > > > > > > > > here
>> > > > > > > > > > >> is
>> > > > > > > > > > >> > that instead of letting each replica doing log
>> > > compaction
>> > > > > > > > > > independently,
>> > > > > > > > > > >> > we'll have the leader to decide upon which offset
>> to
>> > > > compact
>> > > > > > to,
>> > > > > > > > and
>> > > > > > > > > > >> > propagate this value to others to follow, in a more
>> > > > > > coordinated
>> > > > > > > > > > manner.
>> > > > > > > > > > >> > Also note when there are new voters joining the
>> quorum
>> > > who
>> > > > > has
>> > > > > > > not
>> > > > > > > > > > >> > replicated up to the dirty-offset, of because of
>> other
>> > > > > issues
>> > > > > > > they
>> > > > > > > > > > >> > truncated their logs to below the dirty-offset,
>> they'd
>> > > > have
>> > > > > to
>> > > > > > > > > > >> re-bootstrap
>> > > > > > > > > > >> > from the beginning, and during this period of time
>> the
>> > > > > leader
>> > > > > > > > > learned
>> > > > > > > > > > >> about
>> > > > > > > > > > >> > this lagging voter would not advance the watermark
>> > (also
>> > > > it
>> > > > > > > would
>> > > > > > > > > not
>> > > > > > > > > > >> > decrement it), and hence not compacting either,
>> until
>> > > the
>> > > > > > > voter(s)
>> > > > > > > > > has
>> > > > > > > > > > >> > caught up to that dirty-offset.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > So back to your example above, before the bootstrap
>> > > voter
>> > > > > gets
>> > > > > > > to
>> > > > > > > > > 300
>> > > > > > > > > > no
>> > > > > > > > > > >> > log compaction would happen on the leader; and
>> until
>> > > later
>> > > > > > when
>> > > > > > > > the
>> > > > > > > > > > >> voter
>> > > > > > > > > > >> > have got to beyond 400 and hence replicated that
>> > > > tombstone,
>> > > > > > the
>> > > > > > > > log
>> > > > > > > > > > >> > compaction would possibly get to that tombstone and
>> > > remove
>> > > > > it.
>> > > > > > > Say
>> > > > > > > > > > >> later it
>> > > > > > > > > > >> > the leader's log compaction reaches 500, it can
>> send
>> > > this
>> > > > > back
>> > > > > > > to
>> > > > > > > > > the
>> > > > > > > > > > >> voter
>> > > > > > > > > > >> > who can then also compact locally up to 500.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > > > > 105. Quorum State: In addition to VotedId,
>> do we
>> > > > need
>> > > > > > the
>> > > > > > > > > epoch
>> > > > > > > > > > >> > > > > corresponding to VotedId? Over time, the same
>> > > broker
>> > > > > Id
>> > > > > > > > could
>> > > > > > > > > be
>> > > > > > > > > > >> > voted
>> > > > > > > > > > >> > > in
>> > > > > > > > > > >> > > > > different generations with different epoch.
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > Hmm, this is a good point. Originally I think
>> the
>> > > > > > > > "LeaderEpoch"
>> > > > > > > > > > >> field
>> > > > > > > > > > >> > in
>> > > > > > > > > > >> > > > that file is corresponding to the "latest known
>> > > leader
>> > > > > > > epoch",
>> > > > > > > > > not
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > > "current leader epoch". For example, if the
>> > current
>> > > > > epoch
>> > > > > > is
>> > > > > > > > N,
>> > > > > > > > > > and
>> > > > > > > > > > >> > then
>> > > > > > > > > > >> > > a
>> > > > > > > > > > >> > > > vote-request with epoch N+1 is received and the
>> > > voter
>> > > > > > > granted
>> > > > > > > > > the
>> > > > > > > > > > >> vote
>> > > > > > > > > > >> > > for
>> > > > > > > > > > >> > > > it, then it means for this voter it knows the
>> > > "latest
>> > > > > > epoch"
>> > > > > > > > is
>> > > > > > > > > N
>> > > > > > > > > > +
>> > > > > > > > > > >> 1
>> > > > > > > > > > >> > > > although it is unknown if that sending
>> candidate
>> > > will
>> > > > > > indeed
>> > > > > > > > > > become
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > new
>> > > > > > > > > > >> > > > leader (which would only be notified via
>> > > begin-quorum
>> > > > > > > > request).
>> > > > > > > > > > >> > However,
>> > > > > > > > > > >> > > > when persisting the quorum state, we would
>> encode
>> > > > > > > leader-epoch
>> > > > > > > > > to
>> > > > > > > > > > >> N+1,
>> > > > > > > > > > >> > > > while the leaderId to be the older leader.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > But now thinking about this a bit more, I feel
>> we
>> > > > should
>> > > > > > use
>> > > > > > > > two
>> > > > > > > > > > >> > separate
>> > > > > > > > > > >> > > > epochs, one for the "lates known" and one for
>> the
>> > > > > > "current"
>> > > > > > > to
>> > > > > > > > > > pair
>> > > > > > > > > > >> > with
>> > > > > > > > > > >> > > > the leaderId. I will update the wiki page.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > Hmm, it's kind of weird to bump up the leader
>> epoch
>> > > > before
>> > > > > > the
>> > > > > > > > new
>> > > > > > > > > > >> leader
>> > > > > > > > > > >> > > is actually elected, right.
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
>> > > > > > FetchQuorumRecords
>> > > > > > > > API
>> > > > > > > > > to
>> > > > > > > > > > >> > > indicate
>> > > > > > > > > > >> > > > > that the follower has fetched from an invalid
>> > > offset
>> > > > > and
>> > > > > > > > > should
>> > > > > > > > > > >> > > truncate
>> > > > > > > > > > >> > > > to
>> > > > > > > > > > >> > > > > the offset/epoch indicated in the response."
>> > > > Observers
>> > > > > > > can't
>> > > > > > > > > > >> truncate
>> > > > > > > > > > >> > > > their
>> > > > > > > > > > >> > > > > logs. What should they do with
>> > > OFFSET_OUT_OF_RANGE?
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > I'm not sure if I understand your question?
>> > > Observers
>> > > > > > should
>> > > > > > > > > still
>> > > > > > > > > > >> be
>> > > > > > > > > > >> > > able
>> > > > > > > > > > >> > > > to truncate their logs as well.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > Hmm, I thought only the quorum nodes have local
>> logs
>> > > and
>> > > > > > > > observers
>> > > > > > > > > > >> don't?
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > > 107. "The leader will continue sending
>> > > > BeginQuorumEpoch
>> > > > > to
>> > > > > > > > each
>> > > > > > > > > > >> known
>> > > > > > > > > > >> > > > voter
>> > > > > > > > > > >> > > > > until it has received its endorsement." If a
>> > voter
>> > > > is
>> > > > > > down
>> > > > > > > > > for a
>> > > > > > > > > > >> long
>> > > > > > > > > > >> > > > time,
>> > > > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add
>> > unnecessary
>> > > > > > > overhead.
>> > > > > > > > > > >> > Similarly,
>> > > > > > > > > > >> > > > if a
>> > > > > > > > > > >> > > > > follower stops sending FetchQuorumRecords,
>> does
>> > > the
>> > > > > > leader
>> > > > > > > > > keep
>> > > > > > > > > > >> > sending
>> > > > > > > > > > >> > > > > BeginQuorumEpoch?
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good
>> point.
>> > > The
>> > > > > > > > > > >> > begin-quorum-epoch
>> > > > > > > > > > >> > > > request is for voters to quickly get the new
>> > leader
>> > > > > > > > information;
>> > > > > > > > > > >> > however
>> > > > > > > > > > >> > > > even if they do not get them they can still
>> > > eventually
>> > > > > > learn
>> > > > > > > > > about
>> > > > > > > > > > >> that
>> > > > > > > > > > >> > > > from others via gossiping FindQuorum. I think
>> we
>> > can
>> > > > > > adjust
>> > > > > > > > the
>> > > > > > > > > > >> logic
>> > > > > > > > > > >> > to
>> > > > > > > > > > >> > > > e.g. exponential back-off or with a limited
>> > > > num.retries.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Regarding FetchQuorumRecords: if the follower
>> > sends
>> > > > > > > > > > >> FetchQuorumRecords
>> > > > > > > > > > >> > > > already, it means that follower already knows
>> that
>> > > the
>> > > > > > > broker
>> > > > > > > > is
>> > > > > > > > > > the
>> > > > > > > > > > >> > > > leader, and hence we can stop retrying
>> > > > BeginQuorumEpoch;
>> > > > > > > > however
>> > > > > > > > > > it
>> > > > > > > > > > >> is
>> > > > > > > > > > >> > > > possible that after a follower sends
>> > > > FetchQuorumRecords
>> > > > > > > > already,
>> > > > > > > > > > >> > suddenly
>> > > > > > > > > > >> > > > it stops send it (possibly because it learned
>> > about
>> > > a
>> > > > > > higher
>> > > > > > > > > epoch
>> > > > > > > > > > >> > > leader),
>> > > > > > > > > > >> > > > and hence this broker may be a "zombie" leader
>> and
>> > > we
>> > > > > > > propose
>> > > > > > > > to
>> > > > > > > > > > use
>> > > > > > > > > > >> > the
>> > > > > > > > > > >> > > > fetch.timeout to let the leader to try to
>> verify
>> > if
>> > > it
>> > > > > has
>> > > > > > > > > already
>> > > > > > > > > > >> been
>> > > > > > > > > > >> > > > stale.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > It just seems that we should handle these two
>> cases
>> > > in a
>> > > > > > > > > consistent
>> > > > > > > > > > >> way?
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > Yes I agree, on the leader's side, the
>> > > > FetchQuorumRecords
>> > > > > > > from a
>> > > > > > > > > > >> follower
>> > > > > > > > > > >> > could mean that we no longer needs to send
>> > > > BeginQuorumEpoch
>> > > > > > > > anymore
>> > > > > > > > > > ---
>> > > > > > > > > > >> and
>> > > > > > > > > > >> > it is already part of our current implementations
>> in
>> > > > > > > > > > >> >
>> > > https://github.com/confluentinc/kafka/commits/kafka-raft
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > > Thanks,
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > Jun
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > Jun
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang
>> Wang <
>> > > > > > > > > > wangguoz@gmail.com
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > > > wrote:
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > > Hello Leonard,
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > Thanks for your comments, I'm relying in
>> line
>> > > > below:
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang
>> (Leonard)
>> > > Ge
>> > > > <
>> > > > > > > > > > >> > wge@confluent.io>
>> > > > > > > > > > >> > > > > > wrote:
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > > Hi Kafka developers,
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > > It's great to see this proposal and it
>> took
>> > me
>> > > > > some
>> > > > > > > time
>> > > > > > > > > to
>> > > > > > > > > > >> > finish
>> > > > > > > > > > >> > > > > > reading
>> > > > > > > > > > >> > > > > > > it.
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > > And I have the following questions about
>> the
>> > > > > > Proposal:
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > >    - How do we plan to test this design
>> to
>> > > > ensure
>> > > > > > its
>> > > > > > > > > > >> > correctness?
>> > > > > > > > > > >> > > Or
>> > > > > > > > > > >> > > > > > more
>> > > > > > > > > > >> > > > > > >    broadly, how do we ensure that our new
>> > > ‘pull’
>> > > > > > based
>> > > > > > > > > model
>> > > > > > > > > > >> is
>> > > > > > > > > > >> > > > > > functional
>> > > > > > > > > > >> > > > > > > and
>> > > > > > > > > > >> > > > > > >    correct given that it is different
>> from
>> > the
>> > > > > > > original
>> > > > > > > > > RAFT
>> > > > > > > > > > >> > > > > > implementation
>> > > > > > > > > > >> > > > > > >    which has formal proof of correctness?
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > We have two planned verifications on the
>> > > > correctness
>> > > > > > and
>> > > > > > > > > > >> liveness
>> > > > > > > > > > >> > of
>> > > > > > > > > > >> > > > the
>> > > > > > > > > > >> > > > > > design. One is via model verification
>> (TLA+)
>> > > > > > > > > > >> > > > > >
>> > > > https://github.com/guozhangwang/kafka-specification
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > Another is via the concurrent simulation
>> tests
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> >
>> > > > > > > > > > >>
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > >    - Have we considered any sensible
>> defaults
>> > > for
>> > > > > the
>> > > > > > > > > > >> > configuration,
>> > > > > > > > > > >> > > > i.e.
>> > > > > > > > > > >> > > > > > >    all the election timeout, fetch time
>> out,
>> > > > etc.?
>> > > > > > Or
>> > > > > > > we
>> > > > > > > > > > want
>> > > > > > > > > > >> to
>> > > > > > > > > > >> > > > leave
>> > > > > > > > > > >> > > > > > > this to
>> > > > > > > > > > >> > > > > > >    a later stage when we do the
>> performance
>> > > > > testing,
>> > > > > > > > etc.
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > This is a good question, the reason we did
>> not
>> > > set
>> > > > > any
>> > > > > > > > > default
>> > > > > > > > > > >> > values
>> > > > > > > > > > >> > > > for
>> > > > > > > > > > >> > > > > > the timeout configurations is that we
>> think it
>> > > may
>> > > > > > take
>> > > > > > > > some
>> > > > > > > > > > >> > > > benchmarking
>> > > > > > > > > > >> > > > > > experiments to get these defaults right.
>> Some
>> > > > > > high-level
>> > > > > > > > > > >> principles
>> > > > > > > > > > >> > > to
>> > > > > > > > > > >> > > > > > consider: 1) the fetch.timeout should be
>> > around
>> > > > the
>> > > > > > same
>> > > > > > > > > scale
>> > > > > > > > > > >> with
>> > > > > > > > > > >> > > zk
>> > > > > > > > > > >> > > > > > session timeout, which is now 18 seconds by
>> > > > default
>> > > > > --
>> > > > > > > in
>> > > > > > > > > > >> practice
>> > > > > > > > > > >> > > > we've
>> > > > > > > > > > >> > > > > > seen unstable networks having more than 10
>> > secs
>> > > of
>> > > > > > > > transient
>> > > > > > > > > > >> > > > > connectivity,
>> > > > > > > > > > >> > > > > > 2) the election.timeout, however, should be
>> > > > smaller
>> > > > > > than
>> > > > > > > > the
>> > > > > > > > > > >> fetch
>> > > > > > > > > > >> > > > > timeout
>> > > > > > > > > > >> > > > > > as is also suggested as a practical
>> > optimization
>> > > > in
>> > > > > > > > > > literature:
>> > > > > > > > > > >> > > > > >
>> > > > > > > > >
>> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > Some more discussions can be found here:
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > >
>> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > >    - Have we considered piggybacking
>> > > > > > > `BeginQuorumEpoch`
>> > > > > > > > > with
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > `
>> > > > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be
>> missing
>> > > > > something
>> > > > > > > > > obvious
>> > > > > > > > > > >> but
>> > > > > > > > > > >> > I
>> > > > > > > > > > >> > > am
>> > > > > > > > > > >> > > > > > just
>> > > > > > > > > > >> > > > > > >    wondering why don’t we just use the
>> > > > > `FindQuorum`
>> > > > > > > and
>> > > > > > > > > > >> > > > > > > `FetchQuorumRecords`
>> > > > > > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch`
>> > API?
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from
>> > > leader
>> > > > > ->
>> > > > > > > > other
>> > > > > > > > > > >> voter
>> > > > > > > > > > >> > > > > > followers, while FindQuorum / Fetch are
>> sent
>> > > from
>> > > > > > > follower
>> > > > > > > > > to
>> > > > > > > > > > >> > leader.
>> > > > > > > > > > >> > > > > > Arguably one can eventually realize the new
>> > > leader
>> > > > > and
>> > > > > > > > epoch
>> > > > > > > > > > via
>> > > > > > > > > > >> > > > > gossiping
>> > > > > > > > > > >> > > > > > FindQuorum, but that could in practice
>> > require a
>> > > > > long
>> > > > > > > > delay.
>> > > > > > > > > > >> > Having a
>> > > > > > > > > > >> > > > > > leader -> other voters request helps the
>> new
>> > > > leader
>> > > > > > > epoch
>> > > > > > > > to
>> > > > > > > > > > be
>> > > > > > > > > > >> > > > > propagated
>> > > > > > > > > > >> > > > > > faster under a pull model.
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > >    - And about the `FetchQuorumRecords`
>> > > response
>> > > > > > > schema,
>> > > > > > > > > in
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > > > `Records`
>> > > > > > > > > > >> > > > > > >    field of the response, is it just one
>> > > record
>> > > > or
>> > > > > > all
>> > > > > > > > the
>> > > > > > > > > > >> > records
>> > > > > > > > > > >> > > > > > starting
>> > > > > > > > > > >> > > > > > >    from the FetchOffset? It seems a lot
>> more
>> > > > > > efficient
>> > > > > > > > if
>> > > > > > > > > we
>> > > > > > > > > > >> sent
>> > > > > > > > > > >> > > all
>> > > > > > > > > > >> > > > > the
>> > > > > > > > > > >> > > > > > >    records during the bootstrapping of
>> the
>> > > > > brokers.
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is
>> > just
>> > > > the
>> > > > > > > > > starting
>> > > > > > > > > > >> > offset
>> > > > > > > > > > >> > > of
>> > > > > > > > > > >> > > > > the
>> > > > > > > > > > >> > > > > > batch of records.
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > >    - Regarding the disruptive broker
>> issues,
>> > > > does
>> > > > > > our
>> > > > > > > > pull
>> > > > > > > > > > >> based
>> > > > > > > > > > >> > > > model
>> > > > > > > > > > >> > > > > > >    suffer from it? If so, have we
>> considered
>> > > the
>> > > > > > > > Pre-Vote
>> > > > > > > > > > >> stage?
>> > > > > > > > > > >> > If
>> > > > > > > > > > >> > > > > not,
>> > > > > > > > > > >> > > > > > > why?
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > The disruptive broker is stated in the
>> > original
>> > > > Raft
>> > > > > > > paper
>> > > > > > > > > > >> which is
>> > > > > > > > > > >> > > the
>> > > > > > > > > > >> > > > > > result of the push model design. Our
>> analysis
>> > > > showed
>> > > > > > > that
>> > > > > > > > > with
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > pull
>> > > > > > > > > > >> > > > > > model it is no longer an issue.
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > > Thanks a lot for putting this up, and I
>> hope
>> > > > that
>> > > > > my
>> > > > > > > > > > questions
>> > > > > > > > > > >> > can
>> > > > > > > > > > >> > > be
>> > > > > > > > > > >> > > > > of
>> > > > > > > > > > >> > > > > > > some value to make this KIP better.
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > > Hope to hear from you soon!
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > > Best wishes,
>> > > > > > > > > > >> > > > > > > Leonard
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin
>> > McCabe <
>> > > > > > > > > > >> cmccabe@apache.org
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > > > wrote:
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > > > Hi Jason,
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > It's amazing to see this coming
>> together
>> > :)
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > I haven't had a chance to read in
>> detail,
>> > > but
>> > > > I
>> > > > > > read
>> > > > > > > > the
>> > > > > > > > > > >> > outline
>> > > > > > > > > > >> > > > and
>> > > > > > > > > > >> > > > > a
>> > > > > > > > > > >> > > > > > > few
>> > > > > > > > > > >> > > > > > > > things jumped out at me.
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > First, for every epoch that is 32 bits
>> > > rather
>> > > > > than
>> > > > > > > > 64, I
>> > > > > > > > > > >> sort
>> > > > > > > > > > >> > of
>> > > > > > > > > > >> > > > > wonder
>> > > > > > > > > > >> > > > > > > if
>> > > > > > > > > > >> > > > > > > > that's a good long-term choice.  I keep
>> > > > reading
>> > > > > > > about
>> > > > > > > > > > stuff
>> > > > > > > > > > >> > like
>> > > > > > > > > > >> > > > > this:
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
>> > > > > > > > .
>> > > > > > > > > > >> > > Obviously,
>> > > > > > > > > > >> > > > > > that
>> > > > > > > > > > >> > > > > > > > JIRA is about zxid, which increments
>> much
>> > > > faster
>> > > > > > > than
>> > > > > > > > we
>> > > > > > > > > > >> expect
>> > > > > > > > > > >> > > > these
>> > > > > > > > > > >> > > > > > > > leader epochs to, but it would still be
>> > good
>> > > > to
>> > > > > > see
>> > > > > > > > some
>> > > > > > > > > > >> rough
>> > > > > > > > > > >> > > > > > > calculations
>> > > > > > > > > > >> > > > > > > > about how long 32 bits (or really, 31
>> > bits)
>> > > > will
>> > > > > > > last
>> > > > > > > > us
>> > > > > > > > > > in
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > > cases
>> > > > > > > > > > >> > > > > > > where
>> > > > > > > > > > >> > > > > > > > we're using it, and what the space
>> savings
>> > > > we're
>> > > > > > > > getting
>> > > > > > > > > > >> really
>> > > > > > > > > > >> > > is.
>> > > > > > > > > > >> > > > > It
>> > > > > > > > > > >> > > > > > > > seems like in most cases the tradeoff
>> may
>> > > not
>> > > > be
>> > > > > > > worth
>> > > > > > > > > it?
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > Another thing I've been thinking about
>> is
>> > > how
>> > > > we
>> > > > > > do
>> > > > > > > > > > >> > > > bootstrapping.  I
>> > > > > > > > > > >> > > > > > > > would prefer to be in a world where
>> > > > formatting a
>> > > > > > new
>> > > > > > > > > Kafka
>> > > > > > > > > > >> node
>> > > > > > > > > > >> > > > was a
>> > > > > > > > > > >> > > > > > > first
>> > > > > > > > > > >> > > > > > > > class operation explicitly initiated by
>> > the
>> > > > > admin,
>> > > > > > > > > rather
>> > > > > > > > > > >> than
>> > > > > > > > > > >> > > > > > something
>> > > > > > > > > > >> > > > > > > > that happened implicitly when you
>> started
>> > up
>> > > > the
>> > > > > > > > broker
>> > > > > > > > > > and
>> > > > > > > > > > >> > > things
>> > > > > > > > > > >> > > > > > > "looked
>> > > > > > > > > > >> > > > > > > > blank."
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > The first problem is that things can
>> "look
>> > > > > blank"
>> > > > > > > > > > >> accidentally
>> > > > > > > > > > >> > if
>> > > > > > > > > > >> > > > the
>> > > > > > > > > > >> > > > > > > > storage system is having a bad day.
>> > Clearly
>> > > > in
>> > > > > > the
>> > > > > > > > > > non-Raft
>> > > > > > > > > > >> > > world,
>> > > > > > > > > > >> > > > > > this
>> > > > > > > > > > >> > > > > > > > leads to data loss if the broker that
>> is
>> > > > > > (re)started
>> > > > > > > > > this
>> > > > > > > > > > >> way
>> > > > > > > > > > >> > was
>> > > > > > > > > > >> > > > the
>> > > > > > > > > > >> > > > > > > > leader for some partitions.
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > The second problem is that we have a
>> bit
>> > of
>> > > a
>> > > > > > > chicken
>> > > > > > > > > and
>> > > > > > > > > > >> egg
>> > > > > > > > > > >> > > > problem
>> > > > > > > > > > >> > > > > > > with
>> > > > > > > > > > >> > > > > > > > certain configuration keys.  For
>> example,
>> > > > maybe
>> > > > > > you
>> > > > > > > > want
>> > > > > > > > > > to
>> > > > > > > > > > >> > > > configure
>> > > > > > > > > > >> > > > > > > some
>> > > > > > > > > > >> > > > > > > > connection security settings in your
>> > > cluster,
>> > > > > but
>> > > > > > > you
>> > > > > > > > > > don't
>> > > > > > > > > > >> > want
>> > > > > > > > > > >> > > > them
>> > > > > > > > > > >> > > > > > to
>> > > > > > > > > > >> > > > > > > > ever be stored in a plaintext config
>> file.
>> > > > (For
>> > > > > > > > > example,
>> > > > > > > > > > >> SCRAM
>> > > > > > > > > > >> > > > > > > passwords,
>> > > > > > > > > > >> > > > > > > > etc.)  You could use a broker API to
>> set
>> > the
>> > > > > > > > > > configuration,
>> > > > > > > > > > >> but
>> > > > > > > > > > >> > > > that
>> > > > > > > > > > >> > > > > > > brings
>> > > > > > > > > > >> > > > > > > > up the chicken and egg problem.  The
>> > broker
>> > > > > needs
>> > > > > > to
>> > > > > > > > be
>> > > > > > > > > > >> > > configured
>> > > > > > > > > > >> > > > to
>> > > > > > > > > > >> > > > > > > know
>> > > > > > > > > > >> > > > > > > > how to talk to you, but you need to
>> > > configure
>> > > > it
>> > > > > > > > before
>> > > > > > > > > > you
>> > > > > > > > > > >> can
>> > > > > > > > > > >> > > > talk
>> > > > > > > > > > >> > > > > to
>> > > > > > > > > > >> > > > > > > > it.  Using an external secret manager
>> like
>> > > > Vault
>> > > > > > is
>> > > > > > > > one
>> > > > > > > > > > way
>> > > > > > > > > > >> to
>> > > > > > > > > > >> > > > solve
>> > > > > > > > > > >> > > > > > > this,
>> > > > > > > > > > >> > > > > > > > but not everyone uses an external
>> secret
>> > > > > manager.
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > quorum.voters seems like a similar
>> > > > configuration
>> > > > > > > key.
>> > > > > > > > > In
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > > current
>> > > > > > > > > > >> > > > > > > KIP,
>> > > > > > > > > > >> > > > > > > > this is only read if there is no other
>> > > > > > configuration
>> > > > > > > > > > >> specifying
>> > > > > > > > > > >> > > the
>> > > > > > > > > > >> > > > > > > quorum
>> > > > > > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs
>> > command,
>> > > we
>> > > > > > > > wouldn't
>> > > > > > > > > > need
>> > > > > > > > > > >> > this
>> > > > > > > > > > >> > > > key
>> > > > > > > > > > >> > > > > > > > because we could assume that there was
>> > > always
>> > > > > > quorum
>> > > > > > > > > > >> > information
>> > > > > > > > > > >> > > > > stored
>> > > > > > > > > > >> > > > > > > > locally.
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > best,
>> > > > > > > > > > >> > > > > > > > Colin
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason
>> > > > Gustafson
>> > > > > > > wrote:
>> > > > > > > > > > >> > > > > > > > > Hi All,
>> > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > >> > > > > > > > > I'd like to start a discussion on
>> > KIP-595:
>> > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> >
>> > > > > > > > > > >>
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
>> > > > > > > > > > >> > > > > > > > .
>> > > > > > > > > > >> > > > > > > > > This proposal specifies a Raft
>> protocol
>> > to
>> > > > > > > > ultimately
>> > > > > > > > > > >> replace
>> > > > > > > > > > >> > > > > > Zookeeper
>> > > > > > > > > > >> > > > > > > > > as
>> > > > > > > > > > >> > > > > > > > > documented in KIP-500. Please take a
>> > look
>> > > > and
>> > > > > > > share
>> > > > > > > > > your
>> > > > > > > > > > >> > > > thoughts.
>> > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > >> > > > > > > > > A few minor notes to set the stage a
>> > > little
>> > > > > bit:
>> > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > >> > > > > > > > > - This KIP does not specify the
>> > structure
>> > > of
>> > > > > the
>> > > > > > > > > > messages
>> > > > > > > > > > >> > used
>> > > > > > > > > > >> > > to
>> > > > > > > > > > >> > > > > > > > represent
>> > > > > > > > > > >> > > > > > > > > metadata in Kafka, nor does it
>> specify
>> > the
>> > > > > > > internal
>> > > > > > > > > API
>> > > > > > > > > > >> that
>> > > > > > > > > > >> > > will
>> > > > > > > > > > >> > > > > be
>> > > > > > > > > > >> > > > > > > used
>> > > > > > > > > > >> > > > > > > > > by the controller. Expect these to
>> come
>> > in
>> > > > > later
>> > > > > > > > > > >> proposals.
>> > > > > > > > > > >> > > Here
>> > > > > > > > > > >> > > > we
>> > > > > > > > > > >> > > > > > are
>> > > > > > > > > > >> > > > > > > > > primarily concerned with the
>> replication
>> > > > > > protocol
>> > > > > > > > and
>> > > > > > > > > > >> basic
>> > > > > > > > > > >> > > > > > operational
>> > > > > > > > > > >> > > > > > > > > mechanics.
>> > > > > > > > > > >> > > > > > > > > - We expect many details to change
>> as we
>> > > get
>> > > > > > > closer
>> > > > > > > > to
>> > > > > > > > > > >> > > > integration
>> > > > > > > > > > >> > > > > > with
>> > > > > > > > > > >> > > > > > > > > the controller. Any changes we make
>> will
>> > > be
>> > > > > made
>> > > > > > > > > either
>> > > > > > > > > > as
>> > > > > > > > > > >> > > > > amendments
>> > > > > > > > > > >> > > > > > > to
>> > > > > > > > > > >> > > > > > > > > this KIP or, in the case of larger
>> > > changes,
>> > > > as
>> > > > > > new
>> > > > > > > > > > >> proposals.
>> > > > > > > > > > >> > > > > > > > > - We have a prototype implementation
>> > > which I
>> > > > > > will
>> > > > > > > > put
>> > > > > > > > > > >> online
>> > > > > > > > > > >> > > > within
>> > > > > > > > > > >> > > > > > the
>> > > > > > > > > > >> > > > > > > > > next week which may help in
>> > understanding
>> > > > some
>> > > > > > > > > details.
>> > > > > > > > > > It
>> > > > > > > > > > >> > has
>> > > > > > > > > > >> > > > > > > diverged a
>> > > > > > > > > > >> > > > > > > > > little bit from our proposal, so I am
>> > > > taking a
>> > > > > > > > little
>> > > > > > > > > > >> time to
>> > > > > > > > > > >> > > > bring
>> > > > > > > > > > >> > > > > > it
>> > > > > > > > > > >> > > > > > > in
>> > > > > > > > > > >> > > > > > > > > line. I'll post an update to this
>> thread
>> > > > when
>> > > > > it
>> > > > > > > is
>> > > > > > > > > > >> available
>> > > > > > > > > > >> > > for
>> > > > > > > > > > >> > > > > > > review.
>> > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > >> > > > > > > > > Finally, I want to mention that this
>> > > > proposal
>> > > > > > was
>> > > > > > > > > > drafted
>> > > > > > > > > > >> by
>> > > > > > > > > > >> > > > > myself,
>> > > > > > > > > > >> > > > > > > > Boyang
>> > > > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
>> > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > >> > > > > > > > > Thanks,
>> > > > > > > > > > >> > > > > > > > > Jason
>> > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > > > --
>> > > > > > > > > > >> > > > > > > Leonard Ge
>> > > > > > > > > > >> > > > > > > Software Engineer Intern - Confluent
>> > > > > > > > > > >> > > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > > > --
>> > > > > > > > > > >> > > > > > -- Guozhang
>> > > > > > > > > > >> > > > > >
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > --
>> > > > > > > > > > >> > > > -- Guozhang
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > --
>> > > > > > > > > > >> > -- Guozhang
>> > > > > > > > > > >> >
>> > > > > > > > > > >>
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > --
>> > > > > > > > > -- Guozhang
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > --
>> > > > > > > -- Guozhang
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Unmesh Joshi <un...@gmail.com>.
>>Could you clarify on this question? Which part of the raft group doesn't
>>know about leader dis-connection?
The leader of the controller quorum is partitioned from the controller
cluster, and a different leader is elected for the remaining controller
cluster.
I think there are two things here,
1.  The old leader will not know if it's disconnected from the rest of the
controller quorum cluster unless it receives BeginQuorumEpoch from the new
leader. So it will keep on serving stale metadata to the clients (Brokers,
Producers and Consumers)
2. I assume, the Broker Leases will be managed on the controller quorum
leader. This partitioned leader will keep on tracking broker leases it has,
while the new leader of the quorum will also start managing broker leases.
So while the quorum leader is partitioned, there will be two membership
views of the kafka brokers managed on two leaders.
Unless broker heartbeats are also replicated as part of the Raft log, there
is no way to solve this?
I know LogCabin implementation does replicate client heartbeats. I suspect
that the same issue is there in Zookeeper, which does not replicate client
Ping requests..

Thanks,
Unmesh



On Mon, Jul 27, 2020 at 6:23 AM Boyang Chen <re...@gmail.com>
wrote:

> Thanks for the questions Unmesh!
>
> On Sun, Jul 26, 2020 at 6:18 AM Unmesh Joshi <un...@gmail.com>
> wrote:
>
> > Hi,
> >
> > In the FetchRequest Handling, how to make sure we handle scenarios where
> > the leader might have been disconnected from the cluster, but doesn't
> know
> > yet?
> >
> Could you clarify on this question? Which part of the raft group doesn't
> know about leader
> dis-connection?
>
>
> > As discussed in the Raft Thesis section 6.4, the linearizable semantics
> of
> > read requests is implemented in LogCabin by sending heartbeat to
> followers
> > and waiting till the heartbeats are successful to make sure that the
> leader
> > is still the leader.
> > I think for the controller quorum to make sure none of the consumers get
> > stale data, it's important to have linearizable semantics? In the pull
> > based model, the leader will need to wait for heartbeats from the
> followers
> > before returning each fetch request from the consumer then? Or do we need
> > to introduce some other request?
> > (Zookeeper does not have linearizable semantics for read requests, but as
> > of now all the kafka interactions are through writes and watches).
> >
> > This is a very good question. For our v1 implementation we are not aiming
> to guarantee linearizable read, which
> would be considered as a follow-up effort. Note that today in Kafka there
> is no guarantee on the metadata freshness either,
> so no regression is introduced.
>
>
> > Thanks,
> > Unmesh
> >
> > On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Jason,
> > >
> > > Thanks for the reply.
> > >
> > > 101. Sounds good. Regarding clusterId, I am not sure storing it in the
> > > metadata log is enough. For example, the vote request includes
> clusterId.
> > > So, no one can vote until they know the clusterId. Also, it would be
> > useful
> > > to support the case when a voter completely loses its disk and needs to
> > > recover.
> > >
> > > 210. There is no longer a FindQuorum request. When a follower restarts,
> > how
> > > does it discover the leader? Is that based on DescribeQuorum? It would
> be
> > > useful to document this.
> > >
> > > Jun
> > >
> > > On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Thanks for the questions.
> > > >
> > > > 101. I am treating some of the bootstrapping problems as out of the
> > scope
> > > > of this KIP. I am working on a separate proposal which addresses
> > > > bootstrapping security credentials specifically. Here is a rough
> sketch
> > > of
> > > > how I am seeing it:
> > > >
> > > > 1. Dynamic broker configurations including encrypted passwords will
> be
> > > > persisted in the metadata log and cached in the broker's
> > > `meta.properties`
> > > > file.
> > > > 2. We will provide a tool which allows users to directly override the
> > > > values in `meta.properties` without requiring access to the quorum.
> > This
> > > > can be used to bootstrap the credentials of the voter set itself
> before
> > > the
> > > > cluster has been started.
> > > > 3. Some dynamic config changes will only be allowed when a broker is
> > > > online. For example, changing a truststore password dynamically would
> > > > prevent that broker from being able to start if it were offline when
> > the
> > > > change was made.
> > > > 4. I am still thinking a little bit about SCRAM credentials, but most
> > > > likely they will be handled with an approach similar to
> > > `meta.properties`.
> > > >
> > > > 101.3 As for the question about `clusterId`, I think the way we would
> > do
> > > > this is to have the first elected leader generate a UUID and write it
> > to
> > > > the metadata log. Let me add some detail to the proposal about this.
> > > >
> > > > A few additional answers below:
> > > >
> > > > 203. Yes, that is correct.
> > > >
> > > > 204. That is a good question. What happens in this case is that all
> > > voters
> > > > advance their epoch to the one designated by the candidate even if
> they
> > > > reject its vote request. Assuming the candidate fails to be elected,
> > the
> > > > election will be retried until a leader emerges.
> > > >
> > > > 205. I had some discussion with Colin offline about this problem. I
> > think
> > > > the answer should be "yes," but it probably needs a little more
> > thought.
> > > > Handling JBOD failures is tricky. For an observer, we can replicate
> the
> > > > metadata log from scratch safely in a new log dir. But if the log dir
> > of
> > > a
> > > > voter fails, I do not think it is generally safe to start from an
> empty
> > > > state.
> > > >
> > > > 206. Yes, that is discussed in KIP-631 I believe.
> > > >
> > > > 207. Good suggestion. I will work on this.
> > > >
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Jason,
> > > > >
> > > > > Thanks for the updated KIP. Looks good overall. A few more comments
> > > > below.
> > > > >
> > > > > 101. I still don't see a section on bootstrapping related issues.
> It
> > > > would
> > > > > be useful to document if/how the following is supported.
> > > > > 101.1 Currently, we support auto broker id generation. Is this
> > > supported
> > > > > for bootstrap brokers?
> > > > > 101.2 As Colin mentioned, sometimes we may need to load the
> security
> > > > > credentials to be broker before it can be connected to. Could you
> > > > provide a
> > > > > bit more detail on how this will work?
> > > > > 101.3 Currently, we use ZK to generate clusterId on a new cluster.
> > With
> > > > > Raft, how does every broker generate the same clusterId in a
> > > distributed
> > > > > way?
> > > > >
> > > > > 200. It would be useful to document if the various special offsets
> > (log
> > > > > start offset, recovery point, HWM, etc) for the Raft log are stored
> > in
> > > > the
> > > > > same existing checkpoint files or not.
> > > > > 200.1 Since the Raft log flushes every append, does that allow us
> to
> > > > > recover from a recovery point within the active segment or do we
> > still
> > > > need
> > > > > to scan the full segment including the recovery point? The former
> can
> > > be
> > > > > tricky since multiple records can fall into the same disk page and
> a
> > > > > subsequent flush may corrupt a page with previously flushed
> records.
> > > > >
> > > > > 201. Configurations.
> > > > > 201.1 How do the Raft brokers get security related configs for
> inter
> > > > broker
> > > > > communication? Is that based on the existing
> > > > > inter.broker.security.protocol?
> > > > > 201.2 We have quorum.retry.backoff.max.ms and
> > quorum.retry.backoff.ms,
> > > > but
> > > > > only quorum.election.backoff.max.ms. This seems a bit
> inconsistent.
> > > > >
> > > > > 202. Metrics:
> > > > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
> > > OutboundQueueTimeMs:
> > > > > Are those the same as existing totalTime, requestQueueTime,
> > localTime,
> > > > > responseQueueTime? Could we reuse the existing ones with the tag
> > > > > request=[request-type]?
> > > > > 202.2. Could you explain what InboundChannelSize and
> > > OutboundChannelSize
> > > > > are?
> > > > > 202.3 ElectionLatencyMax/Avg: It seems that both should be
> windowed?
> > > > >
> > > > > 203. Quorum State: I assume that LeaderId will be kept consistently
> > > with
> > > > > LeaderEpoch. For example, if a follower transitions to candidate
> and
> > > > bumps
> > > > > up LeaderEpoch, it will set leaderId to -1 and persist both in the
> > > Quorum
> > > > > state file. Is that correct?
> > > > >
> > > > > 204. I was thinking about a corner case when a Raft broker is
> > > partitioned
> > > > > off. This broker will then be in a continuous loop of bumping up
> the
> > > > leader
> > > > > epoch, but failing to get enough votes. When the partitioning is
> > > removed,
> > > > > this broker's high leader epoch will force a leader election. I
> > assume
> > > > > other Raft brokers can immediately advance their leader epoch
> passing
> > > the
> > > > > already bumped epoch such that leader election won't be delayed. Is
> > > that
> > > > > right?
> > > > >
> > > > > 205. In a JBOD setting, could we use the existing tool to move the
> > Raft
> > > > log
> > > > > from one disk to another?
> > > > >
> > > > > 206. The KIP doesn't mention the local metadata store derived from
> > the
> > > > Raft
> > > > > log. Will that be covered in a separate KIP?
> > > > >
> > > > > 207. Since this is a critical component. Could we add a section on
> > the
> > > > > testing plan for correctness?
> > > > >
> > > > > 208. Performance. Do we plan to do group commit (e.g. buffer
> pending
> > > > > appends during a flush and then flush all accumulated pending
> records
> > > > > together in the next flush) for better throughput?
> > > > >
> > > > > 209. "the leader can actually defer fsync until it knows
> > "quorum.size -
> > > > 1"
> > > > > has get to a certain entry offset." Why is that "quorum.size - 1"
> > > instead
> > > > > of the majority of the quorum?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <
> jason@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Hi All,
> > > > > >
> > > > > > Just a quick update on the proposal. We have decided to move
> quorum
> > > > > > reassignment to a separate KIP:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > > > > > .
> > > > > > The way this ties into cluster bootstrapping is complicated, so
> we
> > > felt
> > > > > we
> > > > > > needed a bit more time for validation. That leaves the core of
> this
> > > > > > proposal as quorum-based replication. If there are no further
> > > comments,
> > > > > we
> > > > > > will plan to start a vote later this week.
> > > > > >
> > > > > > Thanks,
> > > > > > Jason
> > > > > >
> > > > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > @Jun Rao <ju...@gmail.com>
> > > > > > >
> > > > > > > Regarding your comment about log compaction. After some
> > deep-diving
> > > > > into
> > > > > > > this we've decided to propose a new snapshot-based log cleaning
> > > > > mechanism
> > > > > > > which would be used to replace the current compaction mechanism
> > for
> > > > > this
> > > > > > > meta log. A new KIP will be proposed specifically for this
> idea.
> > > > > > >
> > > > > > > All,
> > > > > > >
> > > > > > > I've updated the KIP wiki a bit updating one config "
> > > > > > > election.jitter.max.ms"
> > > > > > > to "election.backoff.max.ms" to make it more clear about the
> > > usage:
> > > > > the
> > > > > > > configured value will be the upper bound of the binary
> > exponential
> > > > > > backoff
> > > > > > > time after a failed election, before starting a new one.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> > > > > reluctanthero104@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for the suggestions Guozhang.
> > > > > > > >
> > > > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hello Boyang,
> > > > > > > > >
> > > > > > > > > Thanks for the updated information. A few questions here:
> > > > > > > > >
> > > > > > > > > 1) Should the quorum-file also update to support
> multi-raft?
> > > > > > > > >
> > > > > > > > > I'm neutral about this, as we don't know yet how the
> > multi-raft
> > > > > > modules
> > > > > > > > would behave. If
> > > > > > > > we have different threads operating different raft groups,
> > > > > > consolidating
> > > > > > > > the `checkpoint` files seems
> > > > > > > > not reasonable. We could always add `multi-quorum-file` later
> > if
> > > > > > > possible.
> > > > > > > >
> > > > > > > > 2) In the previous proposal, there's fields in the
> > > > FetchQuorumRecords
> > > > > > > like
> > > > > > > > > latestDirtyOffset, is that dropped intentionally?
> > > > > > > > >
> > > > > > > > > I dropped the latestDirtyOffset since it is associated with
> > the
> > > > log
> > > > > > > > compaction discussion. This is beyond this KIP scope and we
> > could
> > > > > > > > potentially get a separate KIP to talk about it.
> > > > > > > >
> > > > > > > >
> > > > > > > > > 3) I think we also need to elaborate a bit more details
> > > regarding
> > > > > > when
> > > > > > > to
> > > > > > > > > send metadata request and discover-brokers; currently we
> only
> > > > > > discussed
> > > > > > > > > during bootstrap how these requests would be sent. I think
> > the
> > > > > > > following
> > > > > > > > > scenarios would also need these requests
> > > > > > > > >
> > > > > > > > > 3.a) As long as a broker does not know the current quorum
> > > > > (including
> > > > > > > the
> > > > > > > > > leader and the voters), it should continue periodically ask
> > > other
> > > > > > > brokers
> > > > > > > > > via "metadata.
> > > > > > > > > 3.b) As long as a broker does not know all the current
> quorum
> > > > > voter's
> > > > > > > > > connections, it should continue periodically ask other
> > brokers
> > > > via
> > > > > > > > > "discover-brokers".
> > > > > > > > > 3.c) When the leader's fetch timeout elapsed, it should
> send
> > > > > metadata
> > > > > > > > > request.
> > > > > > > > >
> > > > > > > > > Make sense, will add to the KIP.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > Guozhang
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > > > > > reluctanthero104@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey all,
> > > > > > > > > >
> > > > > > > > > > follow-up on the previous email, we made some more
> updates:
> > > > > > > > > >
> > > > > > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured
> to
> > > use
> > > > > > > > > multi-raft.
> > > > > > > > > >
> > > > > > > > > > 2. We add observer status into the DescribeQuorumResponse
> > as
> > > we
> > > > > see
> > > > > > > it
> > > > > > > > > is a
> > > > > > > > > > low hanging fruit which is very useful for user debugging
> > and
> > > > > > > > > reassignment.
> > > > > > > > > >
> > > > > > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers
> RPC,
> > > > which
> > > > > > is
> > > > > > > > > purely
> > > > > > > > > > in charge of discovering broker connections in a gossip
> > > manner.
> > > > > The
> > > > > > > > > quorum
> > > > > > > > > > leader discovery is piggy-back on the Metadata RPC for
> the
> > > > topic
> > > > > > > > > partition
> > > > > > > > > > leader, which in our case is the single metadata
> partition
> > > for
> > > > > the
> > > > > > > > > version
> > > > > > > > > > one.
> > > > > > > > > >
> > > > > > > > > > Let me know if you have any questions.
> > > > > > > > > >
> > > > > > > > > > Boyang
> > > > > > > > > >
> > > > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > > > > > reluctanthero104@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey all,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the great discussions so far. I'm posting
> some
> > > KIP
> > > > > > > updates
> > > > > > > > > > from
> > > > > > > > > > > our working group discussion:
> > > > > > > > > > >
> > > > > > > > > > > 1. We will be changing the core RPCs from single-raft
> API
> > > to
> > > > > > > > > multi-raft.
> > > > > > > > > > > This means all protocols will be "batch" in the first
> > > > version,
> > > > > > but
> > > > > > > > the
> > > > > > > > > > KIP
> > > > > > > > > > > itself only illustrates the design for a single
> metadata
> > > > topic
> > > > > > > > > partition.
> > > > > > > > > > > The reason is to "keep the door open" for future
> > extensions
> > > > of
> > > > > > this
> > > > > > > > > piece
> > > > > > > > > > > of module such as a sharded controller or general
> quorum
> > > > based
> > > > > > > topic
> > > > > > > > > > > replication, beyond the current Kafka replication
> > protocol.
> > > > > > > > > > >
> > > > > > > > > > > 2. We will piggy-back on the current Kafka Fetch API
> > > instead
> > > > of
> > > > > > > > > inventing
> > > > > > > > > > > a new FetchQuorumRecords RPC. The motivation is about
> the
> > > > same
> > > > > as
> > > > > > > #1
> > > > > > > > as
> > > > > > > > > > > well as making the integration work easier, instead of
> > > > letting
> > > > > > two
> > > > > > > > > > similar
> > > > > > > > > > > RPCs diverge.
> > > > > > > > > > >
> > > > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of only
> > sending
> > > > the
> > > > > > > > request
> > > > > > > > > to
> > > > > > > > > > > the most caught-up voter, we shall broadcast the
> > > information
> > > > to
> > > > > > all
> > > > > > > > > > voters,
> > > > > > > > > > > with a sorted voter list in descending order of their
> > > > > > corresponding
> > > > > > > > > > > replicated offset. In this way, the top voter will
> > become a
> > > > > > > candidate
> > > > > > > > > > > immediately, while the other voters shall wait for an
> > > > > exponential
> > > > > > > > > > back-off
> > > > > > > > > > > to trigger elections, which helps ensure the top voter
> > gets
> > > > > > > elected,
> > > > > > > > > and
> > > > > > > > > > > the election eventually happens when the top voter is
> not
> > > > > > > responsive.
> > > > > > > > > > >
> > > > > > > > > > > Please see the updated KIP and post any questions or
> > > concerns
> > > > > on
> > > > > > > the
> > > > > > > > > > > mailing thread.
> > > > > > > > > > >
> > > > > > > > > > > Boyang
> > > > > > > > > > >
> > > > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > >> Hi, Guozhang and Jason,
> > > > > > > > > > >>
> > > > > > > > > > >> Thanks for the reply. A couple of more replies.
> > > > > > > > > > >>
> > > > > > > > > > >> 102. Still not sure about this. How is the tombstone
> > issue
> > > > > > > addressed
> > > > > > > > > in
> > > > > > > > > > >> the
> > > > > > > > > > >> non-voter and the observer.  They can die at any point
> > and
> > > > > > restart
> > > > > > > > at
> > > > > > > > > an
> > > > > > > > > > >> arbitrary later time, and the advancing of the
> > firstDirty
> > > > > offset
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > >> removal of the tombstone can happen independently.
> > > > > > > > > > >>
> > > > > > > > > > >> 106. I agree that it would be less confusing if we
> used
> > > > > "epoch"
> > > > > > > > > instead
> > > > > > > > > > of
> > > > > > > > > > >> "leader epoch" consistently.
> > > > > > > > > > >>
> > > > > > > > > > >> Jun
> > > > > > > > > > >>
> > > > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >>
> > > > > > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > > > > > > >> >
> > > > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > > > > >> >
> > > > > > > > > > >> > > Hi, Guozhang,
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Thanks for the reply. A few more replies inlined
> > > below.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > > > > > > wangguoz@gmail.com>
> > > > > > > > > > >> wrote:
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > Hello Jun,
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Thanks for your comments! I'm replying inline
> > below:
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> > > > > jun@confluent.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > > > > > > >> > > > > 101.1 Currently, we support auto broker id
> > > > generation.
> > > > > > Is
> > > > > > > > this
> > > > > > > > > > >> > > supported
> > > > > > > > > > >> > > > > for bootstrap brokers?
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > The vote ids would just be the broker ids.
> > > > > > > "bootstrap.servers"
> > > > > > > > > > >> would be
> > > > > > > > > > >> > > > similar to what client configs have today, where
> > > > > > > > "quorum.voters"
> > > > > > > > > > >> would
> > > > > > > > > > >> > be
> > > > > > > > > > >> > > > pre-defined config values.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > My question was on the auto generated broker id.
> > > > > Currently,
> > > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > >> > can
> > > > > > > > > > >> > > choose to have its broker Id auto generated. The
> > > > > generation
> > > > > > is
> > > > > > > > > done
> > > > > > > > > > >> > through
> > > > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not
> > clear
> > > > how
> > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > >> id
> > > > > > > > > > >> > is
> > > > > > > > > > >> > > auto generated. "quorum.voters" also can't be set
> > > > > statically
> > > > > > > if
> > > > > > > > > > broker
> > > > > > > > > > >> > ids
> > > > > > > > > > >> > > are auto generated.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Jason has explained some ideas that we've
> discussed
> > so
> > > > > far,
> > > > > > > the
> > > > > > > > > > >> reason we
> > > > > > > > > > >> > intentional did not include them so far is that we
> > feel
> > > it
> > > > > is
> > > > > > > > > out-side
> > > > > > > > > > >> the
> > > > > > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we
> > > should
> > > > > > > > definitely
> > > > > > > > > > >> > address them though.
> > > > > > > > > > >> >
> > > > > > > > > > >> > On the high-level, our belief is that "joining a
> > quorum"
> > > > and
> > > > > > > > > "joining
> > > > > > > > > > >> (or
> > > > > > > > > > >> > more specifically, registering brokers in) the
> > cluster"
> > > > > would
> > > > > > be
> > > > > > > > > > >> > de-coupled a bit, where the former should be
> completed
> > > > > before
> > > > > > we
> > > > > > > > do
> > > > > > > > > > the
> > > > > > > > > > >> > latter. More specifically, assuming the quorum is
> > > already
> > > > up
> > > > > > and
> > > > > > > > > > >> running,
> > > > > > > > > > >> > after the newly started broker found the leader of
> the
> > > > > quorum
> > > > > > it
> > > > > > > > can
> > > > > > > > > > >> send a
> > > > > > > > > > >> > specific RegisterBroker request including its
> > listener /
> > > > > > > protocol
> > > > > > > > /
> > > > > > > > > > etc,
> > > > > > > > > > >> > and upon handling it the leader can send back the
> > > uniquely
> > > > > > > > generated
> > > > > > > > > > >> broker
> > > > > > > > > > >> > id to the new broker, while also executing the
> > > > > > "startNewBroker"
> > > > > > > > > > >> callback as
> > > > > > > > > > >> > the controller.
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > > 102. Log compaction. One weak spot of log
> > > compaction
> > > > > is
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > >> > > consumer
> > > > > > > > > > >> > > > to
> > > > > > > > > > >> > > > > deal with deletes. When a key is deleted, it's
> > > > > retained
> > > > > > > as a
> > > > > > > > > > >> > tombstone
> > > > > > > > > > >> > > > > first and then physically removed. If a client
> > > > misses
> > > > > > the
> > > > > > > > > > >> tombstone
> > > > > > > > > > >> > > > > (because it's physically removed), it may not
> be
> > > > able
> > > > > to
> > > > > > > > > update
> > > > > > > > > > >> its
> > > > > > > > > > >> > > > > metadata properly. The way we solve this in
> > Kafka
> > > is
> > > > > > based
> > > > > > > > on
> > > > > > > > > a
> > > > > > > > > > >> > > > > configuration (
> log.cleaner.delete.retention.ms)
> > > and
> > > > > we
> > > > > > > > > expect a
> > > > > > > > > > >> > > consumer
> > > > > > > > > > >> > > > > having seen an old key to finish reading the
> > > > deletion
> > > > > > > > > tombstone
> > > > > > > > > > >> > within
> > > > > > > > > > >> > > > that
> > > > > > > > > > >> > > > > time. There is no strong guarantee for that
> > since
> > > a
> > > > > > broker
> > > > > > > > > could
> > > > > > > > > > >> be
> > > > > > > > > > >> > > down
> > > > > > > > > > >> > > > > for a long time. It would be better if we can
> > > have a
> > > > > > more
> > > > > > > > > > reliable
> > > > > > > > > > >> > way
> > > > > > > > > > >> > > of
> > > > > > > > > > >> > > > > dealing with deletes.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > We propose to capture this in the
> > "FirstDirtyOffset"
> > > > > field
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > >> > quorum
> > > > > > > > > > >> > > > record fetch response: the offset is the maximum
> > > > offset
> > > > > > that
> > > > > > > > log
> > > > > > > > > > >> > > compaction
> > > > > > > > > > >> > > > has reached up to. If the follower has fetched
> > > beyond
> > > > > this
> > > > > > > > > offset
> > > > > > > > > > it
> > > > > > > > > > >> > > means
> > > > > > > > > > >> > > > itself is safe hence it has seen all records up
> to
> > > > that
> > > > > > > > offset.
> > > > > > > > > On
> > > > > > > > > > >> > > getting
> > > > > > > > > > >> > > > the response, the follower can then decide if
> its
> > > end
> > > > > > offset
> > > > > > > > > > >> actually
> > > > > > > > > > >> > > below
> > > > > > > > > > >> > > > that dirty offset (and hence may miss some
> > > > tombstones).
> > > > > If
> > > > > > > > > that's
> > > > > > > > > > >> the
> > > > > > > > > > >> > > case:
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log
> > from
> > > > the
> > > > > > very
> > > > > > > > > > >> beginning
> > > > > > > > > > >> > to
> > > > > > > > > > >> > > > catch up.
> > > > > > > > > > >> > > > 2) During that time, it would refrain itself
> from
> > > > > > answering
> > > > > > > > > > >> > > MetadataRequest
> > > > > > > > > > >> > > > from any clients.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > I am not sure that the "FirstDirtyOffset" field
> > fully
> > > > > > > addresses
> > > > > > > > > the
> > > > > > > > > > >> > issue.
> > > > > > > > > > >> > > Currently, the deletion tombstone is not removed
> > > > > immediately
> > > > > > > > > after a
> > > > > > > > > > >> > round
> > > > > > > > > > >> > > of cleaning. It's removed after a delay in a
> > > subsequent
> > > > > > round
> > > > > > > of
> > > > > > > > > > >> > cleaning.
> > > > > > > > > > >> > > Consider an example where a key insertion is at
> > offset
> > > > 200
> > > > > > > and a
> > > > > > > > > > >> deletion
> > > > > > > > > > >> > > tombstone of the key is at 400. Initially,
> > > > > FirstDirtyOffset
> > > > > > is
> > > > > > > > at
> > > > > > > > > > >> 300. A
> > > > > > > > > > >> > > follower/observer fetches from offset 0  and
> fetches
> > > the
> > > > > key
> > > > > > > at
> > > > > > > > > > offset
> > > > > > > > > > >> > 200.
> > > > > > > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset
> is
> > > at
> > > > > 500
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > >> > > tombstone at 400 is physically removed. The
> > > > > > follower/observer
> > > > > > > > > > >> continues
> > > > > > > > > > >> > the
> > > > > > > > > > >> > > fetch, but misses offset 400. It catches all the
> way
> > > to
> > > > > > > > > > >> FirstDirtyOffset
> > > > > > > > > > >> > > and declares its metadata as ready. However, its
> > > > metadata
> > > > > > > could
> > > > > > > > be
> > > > > > > > > > >> stale
> > > > > > > > > > >> > > since it actually misses the deletion of the key.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Yeah good question, I should have put more details
> > in
> > > my
> > > > > > > > > explanation
> > > > > > > > > > >> :)
> > > > > > > > > > >> >
> > > > > > > > > > >> > The idea is that we will adjust the log compaction
> for
> > > > this
> > > > > > raft
> > > > > > > > > based
> > > > > > > > > > >> > metadata log: before more details to be explained,
> > since
> > > > we
> > > > > > have
> > > > > > > > two
> > > > > > > > > > >> types
> > > > > > > > > > >> > of "watermarks" here, whereas in Kafka the watermark
> > > > > indicates
> > > > > > > > where
> > > > > > > > > > >> every
> > > > > > > > > > >> > replica have replicated up to and in Raft the
> > watermark
> > > > > > > indicates
> > > > > > > > > > where
> > > > > > > > > > >> the
> > > > > > > > > > >> > majority of replicas (here only indicating voters of
> > the
> > > > > > quorum,
> > > > > > > > not
> > > > > > > > > > >> > counting observers) have replicated up to, let's
> call
> > > them
> > > > > > Kafka
> > > > > > > > > > >> watermark
> > > > > > > > > > >> > and Raft watermark. For this special log, we would
> > > > maintain
> > > > > > both
> > > > > > > > > > >> > watermarks.
> > > > > > > > > > >> >
> > > > > > > > > > >> > When log compacting on the leader, we would only
> > compact
> > > > up
> > > > > to
> > > > > > > the
> > > > > > > > > > Kafka
> > > > > > > > > > >> > watermark, i.e. if there is at least one voter who
> > have
> > > > not
> > > > > > > > > replicated
> > > > > > > > > > >> an
> > > > > > > > > > >> > entry, it would not be compacted. The "dirty-offset"
> > is
> > > > the
> > > > > > > offset
> > > > > > > > > > that
> > > > > > > > > > >> > we've compacted up to and is communicated to other
> > > voters,
> > > > > and
> > > > > > > the
> > > > > > > > > > other
> > > > > > > > > > >> > voters would also compact up to this value --- i.e.
> > the
> > > > > > > difference
> > > > > > > > > > here
> > > > > > > > > > >> is
> > > > > > > > > > >> > that instead of letting each replica doing log
> > > compaction
> > > > > > > > > > independently,
> > > > > > > > > > >> > we'll have the leader to decide upon which offset to
> > > > compact
> > > > > > to,
> > > > > > > > and
> > > > > > > > > > >> > propagate this value to others to follow, in a more
> > > > > > coordinated
> > > > > > > > > > manner.
> > > > > > > > > > >> > Also note when there are new voters joining the
> quorum
> > > who
> > > > > has
> > > > > > > not
> > > > > > > > > > >> > replicated up to the dirty-offset, of because of
> other
> > > > > issues
> > > > > > > they
> > > > > > > > > > >> > truncated their logs to below the dirty-offset,
> they'd
> > > > have
> > > > > to
> > > > > > > > > > >> re-bootstrap
> > > > > > > > > > >> > from the beginning, and during this period of time
> the
> > > > > leader
> > > > > > > > > learned
> > > > > > > > > > >> about
> > > > > > > > > > >> > this lagging voter would not advance the watermark
> > (also
> > > > it
> > > > > > > would
> > > > > > > > > not
> > > > > > > > > > >> > decrement it), and hence not compacting either,
> until
> > > the
> > > > > > > voter(s)
> > > > > > > > > has
> > > > > > > > > > >> > caught up to that dirty-offset.
> > > > > > > > > > >> >
> > > > > > > > > > >> > So back to your example above, before the bootstrap
> > > voter
> > > > > gets
> > > > > > > to
> > > > > > > > > 300
> > > > > > > > > > no
> > > > > > > > > > >> > log compaction would happen on the leader; and until
> > > later
> > > > > > when
> > > > > > > > the
> > > > > > > > > > >> voter
> > > > > > > > > > >> > have got to beyond 400 and hence replicated that
> > > > tombstone,
> > > > > > the
> > > > > > > > log
> > > > > > > > > > >> > compaction would possibly get to that tombstone and
> > > remove
> > > > > it.
> > > > > > > Say
> > > > > > > > > > >> later it
> > > > > > > > > > >> > the leader's log compaction reaches 500, it can send
> > > this
> > > > > back
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > >> voter
> > > > > > > > > > >> > who can then also compact locally up to 500.
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> > > > > 105. Quorum State: In addition to VotedId, do
> we
> > > > need
> > > > > > the
> > > > > > > > > epoch
> > > > > > > > > > >> > > > > corresponding to VotedId? Over time, the same
> > > broker
> > > > > Id
> > > > > > > > could
> > > > > > > > > be
> > > > > > > > > > >> > voted
> > > > > > > > > > >> > > in
> > > > > > > > > > >> > > > > different generations with different epoch.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > Hmm, this is a good point. Originally I think
> the
> > > > > > > > "LeaderEpoch"
> > > > > > > > > > >> field
> > > > > > > > > > >> > in
> > > > > > > > > > >> > > > that file is corresponding to the "latest known
> > > leader
> > > > > > > epoch",
> > > > > > > > > not
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > "current leader epoch". For example, if the
> > current
> > > > > epoch
> > > > > > is
> > > > > > > > N,
> > > > > > > > > > and
> > > > > > > > > > >> > then
> > > > > > > > > > >> > > a
> > > > > > > > > > >> > > > vote-request with epoch N+1 is received and the
> > > voter
> > > > > > > granted
> > > > > > > > > the
> > > > > > > > > > >> vote
> > > > > > > > > > >> > > for
> > > > > > > > > > >> > > > it, then it means for this voter it knows the
> > > "latest
> > > > > > epoch"
> > > > > > > > is
> > > > > > > > > N
> > > > > > > > > > +
> > > > > > > > > > >> 1
> > > > > > > > > > >> > > > although it is unknown if that sending candidate
> > > will
> > > > > > indeed
> > > > > > > > > > become
> > > > > > > > > > >> the
> > > > > > > > > > >> > > new
> > > > > > > > > > >> > > > leader (which would only be notified via
> > > begin-quorum
> > > > > > > > request).
> > > > > > > > > > >> > However,
> > > > > > > > > > >> > > > when persisting the quorum state, we would
> encode
> > > > > > > leader-epoch
> > > > > > > > > to
> > > > > > > > > > >> N+1,
> > > > > > > > > > >> > > > while the leaderId to be the older leader.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > But now thinking about this a bit more, I feel
> we
> > > > should
> > > > > > use
> > > > > > > > two
> > > > > > > > > > >> > separate
> > > > > > > > > > >> > > > epochs, one for the "lates known" and one for
> the
> > > > > > "current"
> > > > > > > to
> > > > > > > > > > pair
> > > > > > > > > > >> > with
> > > > > > > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > Hmm, it's kind of weird to bump up the leader
> epoch
> > > > before
> > > > > > the
> > > > > > > > new
> > > > > > > > > > >> leader
> > > > > > > > > > >> > > is actually elected, right.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > > > > > FetchQuorumRecords
> > > > > > > > API
> > > > > > > > > to
> > > > > > > > > > >> > > indicate
> > > > > > > > > > >> > > > > that the follower has fetched from an invalid
> > > offset
> > > > > and
> > > > > > > > > should
> > > > > > > > > > >> > > truncate
> > > > > > > > > > >> > > > to
> > > > > > > > > > >> > > > > the offset/epoch indicated in the response."
> > > > Observers
> > > > > > > can't
> > > > > > > > > > >> truncate
> > > > > > > > > > >> > > > their
> > > > > > > > > > >> > > > > logs. What should they do with
> > > OFFSET_OUT_OF_RANGE?
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > I'm not sure if I understand your question?
> > > Observers
> > > > > > should
> > > > > > > > > still
> > > > > > > > > > >> be
> > > > > > > > > > >> > > able
> > > > > > > > > > >> > > > to truncate their logs as well.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > Hmm, I thought only the quorum nodes have local
> logs
> > > and
> > > > > > > > observers
> > > > > > > > > > >> don't?
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > 107. "The leader will continue sending
> > > > BeginQuorumEpoch
> > > > > to
> > > > > > > > each
> > > > > > > > > > >> known
> > > > > > > > > > >> > > > voter
> > > > > > > > > > >> > > > > until it has received its endorsement." If a
> > voter
> > > > is
> > > > > > down
> > > > > > > > > for a
> > > > > > > > > > >> long
> > > > > > > > > > >> > > > time,
> > > > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add
> > unnecessary
> > > > > > > overhead.
> > > > > > > > > > >> > Similarly,
> > > > > > > > > > >> > > > if a
> > > > > > > > > > >> > > > > follower stops sending FetchQuorumRecords,
> does
> > > the
> > > > > > leader
> > > > > > > > > keep
> > > > > > > > > > >> > sending
> > > > > > > > > > >> > > > > BeginQuorumEpoch?
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good
> point.
> > > The
> > > > > > > > > > >> > begin-quorum-epoch
> > > > > > > > > > >> > > > request is for voters to quickly get the new
> > leader
> > > > > > > > information;
> > > > > > > > > > >> > however
> > > > > > > > > > >> > > > even if they do not get them they can still
> > > eventually
> > > > > > learn
> > > > > > > > > about
> > > > > > > > > > >> that
> > > > > > > > > > >> > > > from others via gossiping FindQuorum. I think we
> > can
> > > > > > adjust
> > > > > > > > the
> > > > > > > > > > >> logic
> > > > > > > > > > >> > to
> > > > > > > > > > >> > > > e.g. exponential back-off or with a limited
> > > > num.retries.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Regarding FetchQuorumRecords: if the follower
> > sends
> > > > > > > > > > >> FetchQuorumRecords
> > > > > > > > > > >> > > > already, it means that follower already knows
> that
> > > the
> > > > > > > broker
> > > > > > > > is
> > > > > > > > > > the
> > > > > > > > > > >> > > > leader, and hence we can stop retrying
> > > > BeginQuorumEpoch;
> > > > > > > > however
> > > > > > > > > > it
> > > > > > > > > > >> is
> > > > > > > > > > >> > > > possible that after a follower sends
> > > > FetchQuorumRecords
> > > > > > > > already,
> > > > > > > > > > >> > suddenly
> > > > > > > > > > >> > > > it stops send it (possibly because it learned
> > about
> > > a
> > > > > > higher
> > > > > > > > > epoch
> > > > > > > > > > >> > > leader),
> > > > > > > > > > >> > > > and hence this broker may be a "zombie" leader
> and
> > > we
> > > > > > > propose
> > > > > > > > to
> > > > > > > > > > use
> > > > > > > > > > >> > the
> > > > > > > > > > >> > > > fetch.timeout to let the leader to try to verify
> > if
> > > it
> > > > > has
> > > > > > > > > already
> > > > > > > > > > >> been
> > > > > > > > > > >> > > > stale.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > It just seems that we should handle these two
> cases
> > > in a
> > > > > > > > > consistent
> > > > > > > > > > >> way?
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Yes I agree, on the leader's side, the
> > > > FetchQuorumRecords
> > > > > > > from a
> > > > > > > > > > >> follower
> > > > > > > > > > >> > could mean that we no longer needs to send
> > > > BeginQuorumEpoch
> > > > > > > > anymore
> > > > > > > > > > ---
> > > > > > > > > > >> and
> > > > > > > > > > >> > it is already part of our current implementations in
> > > > > > > > > > >> >
> > > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> > > Thanks,
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Jun
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Jun
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang
> <
> > > > > > > > > > wangguoz@gmail.com
> > > > > > > > > > >> >
> > > > > > > > > > >> > > > wrote:
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > > Hello Leonard,
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Thanks for your comments, I'm relying in
> line
> > > > below:
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang
> (Leonard)
> > > Ge
> > > > <
> > > > > > > > > > >> > wge@confluent.io>
> > > > > > > > > > >> > > > > > wrote:
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > > Hi Kafka developers,
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > It's great to see this proposal and it
> took
> > me
> > > > > some
> > > > > > > time
> > > > > > > > > to
> > > > > > > > > > >> > finish
> > > > > > > > > > >> > > > > > reading
> > > > > > > > > > >> > > > > > > it.
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > And I have the following questions about
> the
> > > > > > Proposal:
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > >    - How do we plan to test this design to
> > > > ensure
> > > > > > its
> > > > > > > > > > >> > correctness?
> > > > > > > > > > >> > > Or
> > > > > > > > > > >> > > > > > more
> > > > > > > > > > >> > > > > > >    broadly, how do we ensure that our new
> > > ‘pull’
> > > > > > based
> > > > > > > > > model
> > > > > > > > > > >> is
> > > > > > > > > > >> > > > > > functional
> > > > > > > > > > >> > > > > > > and
> > > > > > > > > > >> > > > > > >    correct given that it is different from
> > the
> > > > > > > original
> > > > > > > > > RAFT
> > > > > > > > > > >> > > > > > implementation
> > > > > > > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > We have two planned verifications on the
> > > > correctness
> > > > > > and
> > > > > > > > > > >> liveness
> > > > > > > > > > >> > of
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > > > > > > >> > > > > >
> > > > https://github.com/guozhangwang/kafka-specification
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Another is via the concurrent simulation
> tests
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> >
> > > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >    - Have we considered any sensible
> defaults
> > > for
> > > > > the
> > > > > > > > > > >> > configuration,
> > > > > > > > > > >> > > > i.e.
> > > > > > > > > > >> > > > > > >    all the election timeout, fetch time
> out,
> > > > etc.?
> > > > > > Or
> > > > > > > we
> > > > > > > > > > want
> > > > > > > > > > >> to
> > > > > > > > > > >> > > > leave
> > > > > > > > > > >> > > > > > > this to
> > > > > > > > > > >> > > > > > >    a later stage when we do the
> performance
> > > > > testing,
> > > > > > > > etc.
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > This is a good question, the reason we did
> not
> > > set
> > > > > any
> > > > > > > > > default
> > > > > > > > > > >> > values
> > > > > > > > > > >> > > > for
> > > > > > > > > > >> > > > > > the timeout configurations is that we think
> it
> > > may
> > > > > > take
> > > > > > > > some
> > > > > > > > > > >> > > > benchmarking
> > > > > > > > > > >> > > > > > experiments to get these defaults right.
> Some
> > > > > > high-level
> > > > > > > > > > >> principles
> > > > > > > > > > >> > > to
> > > > > > > > > > >> > > > > > consider: 1) the fetch.timeout should be
> > around
> > > > the
> > > > > > same
> > > > > > > > > scale
> > > > > > > > > > >> with
> > > > > > > > > > >> > > zk
> > > > > > > > > > >> > > > > > session timeout, which is now 18 seconds by
> > > > default
> > > > > --
> > > > > > > in
> > > > > > > > > > >> practice
> > > > > > > > > > >> > > > we've
> > > > > > > > > > >> > > > > > seen unstable networks having more than 10
> > secs
> > > of
> > > > > > > > transient
> > > > > > > > > > >> > > > > connectivity,
> > > > > > > > > > >> > > > > > 2) the election.timeout, however, should be
> > > > smaller
> > > > > > than
> > > > > > > > the
> > > > > > > > > > >> fetch
> > > > > > > > > > >> > > > > timeout
> > > > > > > > > > >> > > > > > as is also suggested as a practical
> > optimization
> > > > in
> > > > > > > > > > literature:
> > > > > > > > > > >> > > > > >
> > > > > > > > >
> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Some more discussions can be found here:
> > > > > > > > > > >> > > > > >
> > > > > > > > > >
> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > >    - Have we considered piggybacking
> > > > > > > `BeginQuorumEpoch`
> > > > > > > > > with
> > > > > > > > > > >> the
> > > > > > > > > > >> > `
> > > > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing
> > > > > something
> > > > > > > > > obvious
> > > > > > > > > > >> but
> > > > > > > > > > >> > I
> > > > > > > > > > >> > > am
> > > > > > > > > > >> > > > > > just
> > > > > > > > > > >> > > > > > >    wondering why don’t we just use the
> > > > > `FindQuorum`
> > > > > > > and
> > > > > > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch`
> > API?
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from
> > > leader
> > > > > ->
> > > > > > > > other
> > > > > > > > > > >> voter
> > > > > > > > > > >> > > > > > followers, while FindQuorum / Fetch are sent
> > > from
> > > > > > > follower
> > > > > > > > > to
> > > > > > > > > > >> > leader.
> > > > > > > > > > >> > > > > > Arguably one can eventually realize the new
> > > leader
> > > > > and
> > > > > > > > epoch
> > > > > > > > > > via
> > > > > > > > > > >> > > > > gossiping
> > > > > > > > > > >> > > > > > FindQuorum, but that could in practice
> > require a
> > > > > long
> > > > > > > > delay.
> > > > > > > > > > >> > Having a
> > > > > > > > > > >> > > > > > leader -> other voters request helps the new
> > > > leader
> > > > > > > epoch
> > > > > > > > to
> > > > > > > > > > be
> > > > > > > > > > >> > > > > propagated
> > > > > > > > > > >> > > > > > faster under a pull model.
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > >    - And about the `FetchQuorumRecords`
> > > response
> > > > > > > schema,
> > > > > > > > > in
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > > `Records`
> > > > > > > > > > >> > > > > > >    field of the response, is it just one
> > > record
> > > > or
> > > > > > all
> > > > > > > > the
> > > > > > > > > > >> > records
> > > > > > > > > > >> > > > > > starting
> > > > > > > > > > >> > > > > > >    from the FetchOffset? It seems a lot
> more
> > > > > > efficient
> > > > > > > > if
> > > > > > > > > we
> > > > > > > > > > >> sent
> > > > > > > > > > >> > > all
> > > > > > > > > > >> > > > > the
> > > > > > > > > > >> > > > > > >    records during the bootstrapping of the
> > > > > brokers.
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is
> > just
> > > > the
> > > > > > > > > starting
> > > > > > > > > > >> > offset
> > > > > > > > > > >> > > of
> > > > > > > > > > >> > > > > the
> > > > > > > > > > >> > > > > > batch of records.
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > >    - Regarding the disruptive broker
> issues,
> > > > does
> > > > > > our
> > > > > > > > pull
> > > > > > > > > > >> based
> > > > > > > > > > >> > > > model
> > > > > > > > > > >> > > > > > >    suffer from it? If so, have we
> considered
> > > the
> > > > > > > > Pre-Vote
> > > > > > > > > > >> stage?
> > > > > > > > > > >> > If
> > > > > > > > > > >> > > > > not,
> > > > > > > > > > >> > > > > > > why?
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > The disruptive broker is stated in the
> > original
> > > > Raft
> > > > > > > paper
> > > > > > > > > > >> which is
> > > > > > > > > > >> > > the
> > > > > > > > > > >> > > > > > result of the push model design. Our
> analysis
> > > > showed
> > > > > > > that
> > > > > > > > > with
> > > > > > > > > > >> the
> > > > > > > > > > >> > > pull
> > > > > > > > > > >> > > > > > model it is no longer an issue.
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > > Thanks a lot for putting this up, and I
> hope
> > > > that
> > > > > my
> > > > > > > > > > questions
> > > > > > > > > > >> > can
> > > > > > > > > > >> > > be
> > > > > > > > > > >> > > > > of
> > > > > > > > > > >> > > > > > > some value to make this KIP better.
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > Hope to hear from you soon!
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > Best wishes,
> > > > > > > > > > >> > > > > > > Leonard
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin
> > McCabe <
> > > > > > > > > > >> cmccabe@apache.org
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > > wrote:
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > > Hi Jason,
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > It's amazing to see this coming together
> > :)
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > I haven't had a chance to read in
> detail,
> > > but
> > > > I
> > > > > > read
> > > > > > > > the
> > > > > > > > > > >> > outline
> > > > > > > > > > >> > > > and
> > > > > > > > > > >> > > > > a
> > > > > > > > > > >> > > > > > > few
> > > > > > > > > > >> > > > > > > > things jumped out at me.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > First, for every epoch that is 32 bits
> > > rather
> > > > > than
> > > > > > > > 64, I
> > > > > > > > > > >> sort
> > > > > > > > > > >> > of
> > > > > > > > > > >> > > > > wonder
> > > > > > > > > > >> > > > > > > if
> > > > > > > > > > >> > > > > > > > that's a good long-term choice.  I keep
> > > > reading
> > > > > > > about
> > > > > > > > > > stuff
> > > > > > > > > > >> > like
> > > > > > > > > > >> > > > > this:
> > > > > > > > > > >> > > > > > > >
> > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > > > > > .
> > > > > > > > > > >> > > Obviously,
> > > > > > > > > > >> > > > > > that
> > > > > > > > > > >> > > > > > > > JIRA is about zxid, which increments
> much
> > > > faster
> > > > > > > than
> > > > > > > > we
> > > > > > > > > > >> expect
> > > > > > > > > > >> > > > these
> > > > > > > > > > >> > > > > > > > leader epochs to, but it would still be
> > good
> > > > to
> > > > > > see
> > > > > > > > some
> > > > > > > > > > >> rough
> > > > > > > > > > >> > > > > > > calculations
> > > > > > > > > > >> > > > > > > > about how long 32 bits (or really, 31
> > bits)
> > > > will
> > > > > > > last
> > > > > > > > us
> > > > > > > > > > in
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > cases
> > > > > > > > > > >> > > > > > > where
> > > > > > > > > > >> > > > > > > > we're using it, and what the space
> savings
> > > > we're
> > > > > > > > getting
> > > > > > > > > > >> really
> > > > > > > > > > >> > > is.
> > > > > > > > > > >> > > > > It
> > > > > > > > > > >> > > > > > > > seems like in most cases the tradeoff
> may
> > > not
> > > > be
> > > > > > > worth
> > > > > > > > > it?
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > Another thing I've been thinking about
> is
> > > how
> > > > we
> > > > > > do
> > > > > > > > > > >> > > > bootstrapping.  I
> > > > > > > > > > >> > > > > > > > would prefer to be in a world where
> > > > formatting a
> > > > > > new
> > > > > > > > > Kafka
> > > > > > > > > > >> node
> > > > > > > > > > >> > > > was a
> > > > > > > > > > >> > > > > > > first
> > > > > > > > > > >> > > > > > > > class operation explicitly initiated by
> > the
> > > > > admin,
> > > > > > > > > rather
> > > > > > > > > > >> than
> > > > > > > > > > >> > > > > > something
> > > > > > > > > > >> > > > > > > > that happened implicitly when you
> started
> > up
> > > > the
> > > > > > > > broker
> > > > > > > > > > and
> > > > > > > > > > >> > > things
> > > > > > > > > > >> > > > > > > "looked
> > > > > > > > > > >> > > > > > > > blank."
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > The first problem is that things can
> "look
> > > > > blank"
> > > > > > > > > > >> accidentally
> > > > > > > > > > >> > if
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > > storage system is having a bad day.
> > Clearly
> > > > in
> > > > > > the
> > > > > > > > > > non-Raft
> > > > > > > > > > >> > > world,
> > > > > > > > > > >> > > > > > this
> > > > > > > > > > >> > > > > > > > leads to data loss if the broker that is
> > > > > > (re)started
> > > > > > > > > this
> > > > > > > > > > >> way
> > > > > > > > > > >> > was
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > > leader for some partitions.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > The second problem is that we have a bit
> > of
> > > a
> > > > > > > chicken
> > > > > > > > > and
> > > > > > > > > > >> egg
> > > > > > > > > > >> > > > problem
> > > > > > > > > > >> > > > > > > with
> > > > > > > > > > >> > > > > > > > certain configuration keys.  For
> example,
> > > > maybe
> > > > > > you
> > > > > > > > want
> > > > > > > > > > to
> > > > > > > > > > >> > > > configure
> > > > > > > > > > >> > > > > > > some
> > > > > > > > > > >> > > > > > > > connection security settings in your
> > > cluster,
> > > > > but
> > > > > > > you
> > > > > > > > > > don't
> > > > > > > > > > >> > want
> > > > > > > > > > >> > > > them
> > > > > > > > > > >> > > > > > to
> > > > > > > > > > >> > > > > > > > ever be stored in a plaintext config
> file.
> > > > (For
> > > > > > > > > example,
> > > > > > > > > > >> SCRAM
> > > > > > > > > > >> > > > > > > passwords,
> > > > > > > > > > >> > > > > > > > etc.)  You could use a broker API to set
> > the
> > > > > > > > > > configuration,
> > > > > > > > > > >> but
> > > > > > > > > > >> > > > that
> > > > > > > > > > >> > > > > > > brings
> > > > > > > > > > >> > > > > > > > up the chicken and egg problem.  The
> > broker
> > > > > needs
> > > > > > to
> > > > > > > > be
> > > > > > > > > > >> > > configured
> > > > > > > > > > >> > > > to
> > > > > > > > > > >> > > > > > > know
> > > > > > > > > > >> > > > > > > > how to talk to you, but you need to
> > > configure
> > > > it
> > > > > > > > before
> > > > > > > > > > you
> > > > > > > > > > >> can
> > > > > > > > > > >> > > > talk
> > > > > > > > > > >> > > > > to
> > > > > > > > > > >> > > > > > > > it.  Using an external secret manager
> like
> > > > Vault
> > > > > > is
> > > > > > > > one
> > > > > > > > > > way
> > > > > > > > > > >> to
> > > > > > > > > > >> > > > solve
> > > > > > > > > > >> > > > > > > this,
> > > > > > > > > > >> > > > > > > > but not everyone uses an external secret
> > > > > manager.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > quorum.voters seems like a similar
> > > > configuration
> > > > > > > key.
> > > > > > > > > In
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > current
> > > > > > > > > > >> > > > > > > KIP,
> > > > > > > > > > >> > > > > > > > this is only read if there is no other
> > > > > > configuration
> > > > > > > > > > >> specifying
> > > > > > > > > > >> > > the
> > > > > > > > > > >> > > > > > > quorum
> > > > > > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs
> > command,
> > > we
> > > > > > > > wouldn't
> > > > > > > > > > need
> > > > > > > > > > >> > this
> > > > > > > > > > >> > > > key
> > > > > > > > > > >> > > > > > > > because we could assume that there was
> > > always
> > > > > > quorum
> > > > > > > > > > >> > information
> > > > > > > > > > >> > > > > stored
> > > > > > > > > > >> > > > > > > > locally.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > best,
> > > > > > > > > > >> > > > > > > > Colin
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason
> > > > Gustafson
> > > > > > > wrote:
> > > > > > > > > > >> > > > > > > > > Hi All,
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > I'd like to start a discussion on
> > KIP-595:
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> >
> > > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > > > > > >> > > > > > > > .
> > > > > > > > > > >> > > > > > > > > This proposal specifies a Raft
> protocol
> > to
> > > > > > > > ultimately
> > > > > > > > > > >> replace
> > > > > > > > > > >> > > > > > Zookeeper
> > > > > > > > > > >> > > > > > > > > as
> > > > > > > > > > >> > > > > > > > > documented in KIP-500. Please take a
> > look
> > > > and
> > > > > > > share
> > > > > > > > > your
> > > > > > > > > > >> > > > thoughts.
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > A few minor notes to set the stage a
> > > little
> > > > > bit:
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > - This KIP does not specify the
> > structure
> > > of
> > > > > the
> > > > > > > > > > messages
> > > > > > > > > > >> > used
> > > > > > > > > > >> > > to
> > > > > > > > > > >> > > > > > > > represent
> > > > > > > > > > >> > > > > > > > > metadata in Kafka, nor does it specify
> > the
> > > > > > > internal
> > > > > > > > > API
> > > > > > > > > > >> that
> > > > > > > > > > >> > > will
> > > > > > > > > > >> > > > > be
> > > > > > > > > > >> > > > > > > used
> > > > > > > > > > >> > > > > > > > > by the controller. Expect these to
> come
> > in
> > > > > later
> > > > > > > > > > >> proposals.
> > > > > > > > > > >> > > Here
> > > > > > > > > > >> > > > we
> > > > > > > > > > >> > > > > > are
> > > > > > > > > > >> > > > > > > > > primarily concerned with the
> replication
> > > > > > protocol
> > > > > > > > and
> > > > > > > > > > >> basic
> > > > > > > > > > >> > > > > > operational
> > > > > > > > > > >> > > > > > > > > mechanics.
> > > > > > > > > > >> > > > > > > > > - We expect many details to change as
> we
> > > get
> > > > > > > closer
> > > > > > > > to
> > > > > > > > > > >> > > > integration
> > > > > > > > > > >> > > > > > with
> > > > > > > > > > >> > > > > > > > > the controller. Any changes we make
> will
> > > be
> > > > > made
> > > > > > > > > either
> > > > > > > > > > as
> > > > > > > > > > >> > > > > amendments
> > > > > > > > > > >> > > > > > > to
> > > > > > > > > > >> > > > > > > > > this KIP or, in the case of larger
> > > changes,
> > > > as
> > > > > > new
> > > > > > > > > > >> proposals.
> > > > > > > > > > >> > > > > > > > > - We have a prototype implementation
> > > which I
> > > > > > will
> > > > > > > > put
> > > > > > > > > > >> online
> > > > > > > > > > >> > > > within
> > > > > > > > > > >> > > > > > the
> > > > > > > > > > >> > > > > > > > > next week which may help in
> > understanding
> > > > some
> > > > > > > > > details.
> > > > > > > > > > It
> > > > > > > > > > >> > has
> > > > > > > > > > >> > > > > > > diverged a
> > > > > > > > > > >> > > > > > > > > little bit from our proposal, so I am
> > > > taking a
> > > > > > > > little
> > > > > > > > > > >> time to
> > > > > > > > > > >> > > > bring
> > > > > > > > > > >> > > > > > it
> > > > > > > > > > >> > > > > > > in
> > > > > > > > > > >> > > > > > > > > line. I'll post an update to this
> thread
> > > > when
> > > > > it
> > > > > > > is
> > > > > > > > > > >> available
> > > > > > > > > > >> > > for
> > > > > > > > > > >> > > > > > > review.
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > Finally, I want to mention that this
> > > > proposal
> > > > > > was
> > > > > > > > > > drafted
> > > > > > > > > > >> by
> > > > > > > > > > >> > > > > myself,
> > > > > > > > > > >> > > > > > > > Boyang
> > > > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > >> > > > > > > > > Jason
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > --
> > > > > > > > > > >> > > > > > > Leonard Ge
> > > > > > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > --
> > > > > > > > > > >> > > > > > -- Guozhang
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > --
> > > > > > > > > > >> > > > -- Guozhang
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> > --
> > > > > > > > > > >> > -- Guozhang
> > > > > > > > > > >> >
> > > > > > > > > > >>
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > --
> > > > > > > > > -- Guozhang
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Boyang Chen <re...@gmail.com>.
Thanks for the questions Unmesh!

On Sun, Jul 26, 2020 at 6:18 AM Unmesh Joshi <un...@gmail.com> wrote:

> Hi,
>
> In the FetchRequest Handling, how to make sure we handle scenarios where
> the leader might have been disconnected from the cluster, but doesn't know
> yet?
>
Could you clarify on this question? Which part of the raft group doesn't
know about leader
dis-connection?


> As discussed in the Raft Thesis section 6.4, the linearizable semantics of
> read requests is implemented in LogCabin by sending heartbeat to followers
> and waiting till the heartbeats are successful to make sure that the leader
> is still the leader.
> I think for the controller quorum to make sure none of the consumers get
> stale data, it's important to have linearizable semantics? In the pull
> based model, the leader will need to wait for heartbeats from the followers
> before returning each fetch request from the consumer then? Or do we need
> to introduce some other request?
> (Zookeeper does not have linearizable semantics for read requests, but as
> of now all the kafka interactions are through writes and watches).
>
> This is a very good question. For our v1 implementation we are not aiming
to guarantee linearizable read, which
would be considered as a follow-up effort. Note that today in Kafka there
is no guarantee on the metadata freshness either,
so no regression is introduced.


> Thanks,
> Unmesh
>
> On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Jason,
> >
> > Thanks for the reply.
> >
> > 101. Sounds good. Regarding clusterId, I am not sure storing it in the
> > metadata log is enough. For example, the vote request includes clusterId.
> > So, no one can vote until they know the clusterId. Also, it would be
> useful
> > to support the case when a voter completely loses its disk and needs to
> > recover.
> >
> > 210. There is no longer a FindQuorum request. When a follower restarts,
> how
> > does it discover the leader? Is that based on DescribeQuorum? It would be
> > useful to document this.
> >
> > Jun
> >
> > On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hi Jun,
> > >
> > > Thanks for the questions.
> > >
> > > 101. I am treating some of the bootstrapping problems as out of the
> scope
> > > of this KIP. I am working on a separate proposal which addresses
> > > bootstrapping security credentials specifically. Here is a rough sketch
> > of
> > > how I am seeing it:
> > >
> > > 1. Dynamic broker configurations including encrypted passwords will be
> > > persisted in the metadata log and cached in the broker's
> > `meta.properties`
> > > file.
> > > 2. We will provide a tool which allows users to directly override the
> > > values in `meta.properties` without requiring access to the quorum.
> This
> > > can be used to bootstrap the credentials of the voter set itself before
> > the
> > > cluster has been started.
> > > 3. Some dynamic config changes will only be allowed when a broker is
> > > online. For example, changing a truststore password dynamically would
> > > prevent that broker from being able to start if it were offline when
> the
> > > change was made.
> > > 4. I am still thinking a little bit about SCRAM credentials, but most
> > > likely they will be handled with an approach similar to
> > `meta.properties`.
> > >
> > > 101.3 As for the question about `clusterId`, I think the way we would
> do
> > > this is to have the first elected leader generate a UUID and write it
> to
> > > the metadata log. Let me add some detail to the proposal about this.
> > >
> > > A few additional answers below:
> > >
> > > 203. Yes, that is correct.
> > >
> > > 204. That is a good question. What happens in this case is that all
> > voters
> > > advance their epoch to the one designated by the candidate even if they
> > > reject its vote request. Assuming the candidate fails to be elected,
> the
> > > election will be retried until a leader emerges.
> > >
> > > 205. I had some discussion with Colin offline about this problem. I
> think
> > > the answer should be "yes," but it probably needs a little more
> thought.
> > > Handling JBOD failures is tricky. For an observer, we can replicate the
> > > metadata log from scratch safely in a new log dir. But if the log dir
> of
> > a
> > > voter fails, I do not think it is generally safe to start from an empty
> > > state.
> > >
> > > 206. Yes, that is discussed in KIP-631 I believe.
> > >
> > > 207. Good suggestion. I will work on this.
> > >
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > >
> > >
> > >
> > > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Jason,
> > > >
> > > > Thanks for the updated KIP. Looks good overall. A few more comments
> > > below.
> > > >
> > > > 101. I still don't see a section on bootstrapping related issues. It
> > > would
> > > > be useful to document if/how the following is supported.
> > > > 101.1 Currently, we support auto broker id generation. Is this
> > supported
> > > > for bootstrap brokers?
> > > > 101.2 As Colin mentioned, sometimes we may need to load the security
> > > > credentials to be broker before it can be connected to. Could you
> > > provide a
> > > > bit more detail on how this will work?
> > > > 101.3 Currently, we use ZK to generate clusterId on a new cluster.
> With
> > > > Raft, how does every broker generate the same clusterId in a
> > distributed
> > > > way?
> > > >
> > > > 200. It would be useful to document if the various special offsets
> (log
> > > > start offset, recovery point, HWM, etc) for the Raft log are stored
> in
> > > the
> > > > same existing checkpoint files or not.
> > > > 200.1 Since the Raft log flushes every append, does that allow us to
> > > > recover from a recovery point within the active segment or do we
> still
> > > need
> > > > to scan the full segment including the recovery point? The former can
> > be
> > > > tricky since multiple records can fall into the same disk page and a
> > > > subsequent flush may corrupt a page with previously flushed records.
> > > >
> > > > 201. Configurations.
> > > > 201.1 How do the Raft brokers get security related configs for inter
> > > broker
> > > > communication? Is that based on the existing
> > > > inter.broker.security.protocol?
> > > > 201.2 We have quorum.retry.backoff.max.ms and
> quorum.retry.backoff.ms,
> > > but
> > > > only quorum.election.backoff.max.ms. This seems a bit inconsistent.
> > > >
> > > > 202. Metrics:
> > > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
> > OutboundQueueTimeMs:
> > > > Are those the same as existing totalTime, requestQueueTime,
> localTime,
> > > > responseQueueTime? Could we reuse the existing ones with the tag
> > > > request=[request-type]?
> > > > 202.2. Could you explain what InboundChannelSize and
> > OutboundChannelSize
> > > > are?
> > > > 202.3 ElectionLatencyMax/Avg: It seems that both should be windowed?
> > > >
> > > > 203. Quorum State: I assume that LeaderId will be kept consistently
> > with
> > > > LeaderEpoch. For example, if a follower transitions to candidate and
> > > bumps
> > > > up LeaderEpoch, it will set leaderId to -1 and persist both in the
> > Quorum
> > > > state file. Is that correct?
> > > >
> > > > 204. I was thinking about a corner case when a Raft broker is
> > partitioned
> > > > off. This broker will then be in a continuous loop of bumping up the
> > > leader
> > > > epoch, but failing to get enough votes. When the partitioning is
> > removed,
> > > > this broker's high leader epoch will force a leader election. I
> assume
> > > > other Raft brokers can immediately advance their leader epoch passing
> > the
> > > > already bumped epoch such that leader election won't be delayed. Is
> > that
> > > > right?
> > > >
> > > > 205. In a JBOD setting, could we use the existing tool to move the
> Raft
> > > log
> > > > from one disk to another?
> > > >
> > > > 206. The KIP doesn't mention the local metadata store derived from
> the
> > > Raft
> > > > log. Will that be covered in a separate KIP?
> > > >
> > > > 207. Since this is a critical component. Could we add a section on
> the
> > > > testing plan for correctness?
> > > >
> > > > 208. Performance. Do we plan to do group commit (e.g. buffer pending
> > > > appends during a flush and then flush all accumulated pending records
> > > > together in the next flush) for better throughput?
> > > >
> > > > 209. "the leader can actually defer fsync until it knows
> "quorum.size -
> > > 1"
> > > > has get to a certain entry offset." Why is that "quorum.size - 1"
> > instead
> > > > of the majority of the quorum?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <ja...@confluent.io>
> > > > wrote:
> > > >
> > > > > Hi All,
> > > > >
> > > > > Just a quick update on the proposal. We have decided to move quorum
> > > > > reassignment to a separate KIP:
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > > > > .
> > > > > The way this ties into cluster bootstrapping is complicated, so we
> > felt
> > > > we
> > > > > needed a bit more time for validation. That leaves the core of this
> > > > > proposal as quorum-based replication. If there are no further
> > comments,
> > > > we
> > > > > will plan to start a vote later this week.
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > @Jun Rao <ju...@gmail.com>
> > > > > >
> > > > > > Regarding your comment about log compaction. After some
> deep-diving
> > > > into
> > > > > > this we've decided to propose a new snapshot-based log cleaning
> > > > mechanism
> > > > > > which would be used to replace the current compaction mechanism
> for
> > > > this
> > > > > > meta log. A new KIP will be proposed specifically for this idea.
> > > > > >
> > > > > > All,
> > > > > >
> > > > > > I've updated the KIP wiki a bit updating one config "
> > > > > > election.jitter.max.ms"
> > > > > > to "election.backoff.max.ms" to make it more clear about the
> > usage:
> > > > the
> > > > > > configured value will be the upper bound of the binary
> exponential
> > > > > backoff
> > > > > > time after a failed election, before starting a new one.
> > > > > >
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> > > > reluctanthero104@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Thanks for the suggestions Guozhang.
> > > > > > >
> > > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hello Boyang,
> > > > > > > >
> > > > > > > > Thanks for the updated information. A few questions here:
> > > > > > > >
> > > > > > > > 1) Should the quorum-file also update to support multi-raft?
> > > > > > > >
> > > > > > > > I'm neutral about this, as we don't know yet how the
> multi-raft
> > > > > modules
> > > > > > > would behave. If
> > > > > > > we have different threads operating different raft groups,
> > > > > consolidating
> > > > > > > the `checkpoint` files seems
> > > > > > > not reasonable. We could always add `multi-quorum-file` later
> if
> > > > > > possible.
> > > > > > >
> > > > > > > 2) In the previous proposal, there's fields in the
> > > FetchQuorumRecords
> > > > > > like
> > > > > > > > latestDirtyOffset, is that dropped intentionally?
> > > > > > > >
> > > > > > > > I dropped the latestDirtyOffset since it is associated with
> the
> > > log
> > > > > > > compaction discussion. This is beyond this KIP scope and we
> could
> > > > > > > potentially get a separate KIP to talk about it.
> > > > > > >
> > > > > > >
> > > > > > > > 3) I think we also need to elaborate a bit more details
> > regarding
> > > > > when
> > > > > > to
> > > > > > > > send metadata request and discover-brokers; currently we only
> > > > > discussed
> > > > > > > > during bootstrap how these requests would be sent. I think
> the
> > > > > > following
> > > > > > > > scenarios would also need these requests
> > > > > > > >
> > > > > > > > 3.a) As long as a broker does not know the current quorum
> > > > (including
> > > > > > the
> > > > > > > > leader and the voters), it should continue periodically ask
> > other
> > > > > > brokers
> > > > > > > > via "metadata.
> > > > > > > > 3.b) As long as a broker does not know all the current quorum
> > > > voter's
> > > > > > > > connections, it should continue periodically ask other
> brokers
> > > via
> > > > > > > > "discover-brokers".
> > > > > > > > 3.c) When the leader's fetch timeout elapsed, it should send
> > > > metadata
> > > > > > > > request.
> > > > > > > >
> > > > > > > > Make sense, will add to the KIP.
> > > > > > >
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > > > > reluctanthero104@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey all,
> > > > > > > > >
> > > > > > > > > follow-up on the previous email, we made some more updates:
> > > > > > > > >
> > > > > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to
> > use
> > > > > > > > multi-raft.
> > > > > > > > >
> > > > > > > > > 2. We add observer status into the DescribeQuorumResponse
> as
> > we
> > > > see
> > > > > > it
> > > > > > > > is a
> > > > > > > > > low hanging fruit which is very useful for user debugging
> and
> > > > > > > > reassignment.
> > > > > > > > >
> > > > > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC,
> > > which
> > > > > is
> > > > > > > > purely
> > > > > > > > > in charge of discovering broker connections in a gossip
> > manner.
> > > > The
> > > > > > > > quorum
> > > > > > > > > leader discovery is piggy-back on the Metadata RPC for the
> > > topic
> > > > > > > > partition
> > > > > > > > > leader, which in our case is the single metadata partition
> > for
> > > > the
> > > > > > > > version
> > > > > > > > > one.
> > > > > > > > >
> > > > > > > > > Let me know if you have any questions.
> > > > > > > > >
> > > > > > > > > Boyang
> > > > > > > > >
> > > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > > > > reluctanthero104@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey all,
> > > > > > > > > >
> > > > > > > > > > Thanks for the great discussions so far. I'm posting some
> > KIP
> > > > > > updates
> > > > > > > > > from
> > > > > > > > > > our working group discussion:
> > > > > > > > > >
> > > > > > > > > > 1. We will be changing the core RPCs from single-raft API
> > to
> > > > > > > > multi-raft.
> > > > > > > > > > This means all protocols will be "batch" in the first
> > > version,
> > > > > but
> > > > > > > the
> > > > > > > > > KIP
> > > > > > > > > > itself only illustrates the design for a single metadata
> > > topic
> > > > > > > > partition.
> > > > > > > > > > The reason is to "keep the door open" for future
> extensions
> > > of
> > > > > this
> > > > > > > > piece
> > > > > > > > > > of module such as a sharded controller or general quorum
> > > based
> > > > > > topic
> > > > > > > > > > replication, beyond the current Kafka replication
> protocol.
> > > > > > > > > >
> > > > > > > > > > 2. We will piggy-back on the current Kafka Fetch API
> > instead
> > > of
> > > > > > > > inventing
> > > > > > > > > > a new FetchQuorumRecords RPC. The motivation is about the
> > > same
> > > > as
> > > > > > #1
> > > > > > > as
> > > > > > > > > > well as making the integration work easier, instead of
> > > letting
> > > > > two
> > > > > > > > > similar
> > > > > > > > > > RPCs diverge.
> > > > > > > > > >
> > > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of only
> sending
> > > the
> > > > > > > request
> > > > > > > > to
> > > > > > > > > > the most caught-up voter, we shall broadcast the
> > information
> > > to
> > > > > all
> > > > > > > > > voters,
> > > > > > > > > > with a sorted voter list in descending order of their
> > > > > corresponding
> > > > > > > > > > replicated offset. In this way, the top voter will
> become a
> > > > > > candidate
> > > > > > > > > > immediately, while the other voters shall wait for an
> > > > exponential
> > > > > > > > > back-off
> > > > > > > > > > to trigger elections, which helps ensure the top voter
> gets
> > > > > > elected,
> > > > > > > > and
> > > > > > > > > > the election eventually happens when the top voter is not
> > > > > > responsive.
> > > > > > > > > >
> > > > > > > > > > Please see the updated KIP and post any questions or
> > concerns
> > > > on
> > > > > > the
> > > > > > > > > > mailing thread.
> > > > > > > > > >
> > > > > > > > > > Boyang
> > > > > > > > > >
> > > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> Hi, Guozhang and Jason,
> > > > > > > > > >>
> > > > > > > > > >> Thanks for the reply. A couple of more replies.
> > > > > > > > > >>
> > > > > > > > > >> 102. Still not sure about this. How is the tombstone
> issue
> > > > > > addressed
> > > > > > > > in
> > > > > > > > > >> the
> > > > > > > > > >> non-voter and the observer.  They can die at any point
> and
> > > > > restart
> > > > > > > at
> > > > > > > > an
> > > > > > > > > >> arbitrary later time, and the advancing of the
> firstDirty
> > > > offset
> > > > > > and
> > > > > > > > the
> > > > > > > > > >> removal of the tombstone can happen independently.
> > > > > > > > > >>
> > > > > > > > > >> 106. I agree that it would be less confusing if we used
> > > > "epoch"
> > > > > > > > instead
> > > > > > > > > of
> > > > > > > > > >> "leader epoch" consistently.
> > > > > > > > > >>
> > > > > > > > > >> Jun
> > > > > > > > > >>
> > > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > >>
> > > > > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > > > > > >> >
> > > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > > > >> >
> > > > > > > > > >> > > Hi, Guozhang,
> > > > > > > > > >> > >
> > > > > > > > > >> > > Thanks for the reply. A few more replies inlined
> > below.
> > > > > > > > > >> > >
> > > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > > > > > wangguoz@gmail.com>
> > > > > > > > > >> wrote:
> > > > > > > > > >> > >
> > > > > > > > > >> > > > Hello Jun,
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Thanks for your comments! I'm replying inline
> below:
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> > > > jun@confluent.io
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > > > > > >> > > > > 101.1 Currently, we support auto broker id
> > > generation.
> > > > > Is
> > > > > > > this
> > > > > > > > > >> > > supported
> > > > > > > > > >> > > > > for bootstrap brokers?
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > The vote ids would just be the broker ids.
> > > > > > "bootstrap.servers"
> > > > > > > > > >> would be
> > > > > > > > > >> > > > similar to what client configs have today, where
> > > > > > > "quorum.voters"
> > > > > > > > > >> would
> > > > > > > > > >> > be
> > > > > > > > > >> > > > pre-defined config values.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > My question was on the auto generated broker id.
> > > > Currently,
> > > > > > the
> > > > > > > > > broker
> > > > > > > > > >> > can
> > > > > > > > > >> > > choose to have its broker Id auto generated. The
> > > > generation
> > > > > is
> > > > > > > > done
> > > > > > > > > >> > through
> > > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not
> clear
> > > how
> > > > > the
> > > > > > > > > broker
> > > > > > > > > >> id
> > > > > > > > > >> > is
> > > > > > > > > >> > > auto generated. "quorum.voters" also can't be set
> > > > statically
> > > > > > if
> > > > > > > > > broker
> > > > > > > > > >> > ids
> > > > > > > > > >> > > are auto generated.
> > > > > > > > > >> > >
> > > > > > > > > >> > > Jason has explained some ideas that we've discussed
> so
> > > > far,
> > > > > > the
> > > > > > > > > >> reason we
> > > > > > > > > >> > intentional did not include them so far is that we
> feel
> > it
> > > > is
> > > > > > > > out-side
> > > > > > > > > >> the
> > > > > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we
> > should
> > > > > > > definitely
> > > > > > > > > >> > address them though.
> > > > > > > > > >> >
> > > > > > > > > >> > On the high-level, our belief is that "joining a
> quorum"
> > > and
> > > > > > > > "joining
> > > > > > > > > >> (or
> > > > > > > > > >> > more specifically, registering brokers in) the
> cluster"
> > > > would
> > > > > be
> > > > > > > > > >> > de-coupled a bit, where the former should be completed
> > > > before
> > > > > we
> > > > > > > do
> > > > > > > > > the
> > > > > > > > > >> > latter. More specifically, assuming the quorum is
> > already
> > > up
> > > > > and
> > > > > > > > > >> running,
> > > > > > > > > >> > after the newly started broker found the leader of the
> > > > quorum
> > > > > it
> > > > > > > can
> > > > > > > > > >> send a
> > > > > > > > > >> > specific RegisterBroker request including its
> listener /
> > > > > > protocol
> > > > > > > /
> > > > > > > > > etc,
> > > > > > > > > >> > and upon handling it the leader can send back the
> > uniquely
> > > > > > > generated
> > > > > > > > > >> broker
> > > > > > > > > >> > id to the new broker, while also executing the
> > > > > "startNewBroker"
> > > > > > > > > >> callback as
> > > > > > > > > >> > the controller.
> > > > > > > > > >> >
> > > > > > > > > >> >
> > > > > > > > > >> > >
> > > > > > > > > >> > > > > 102. Log compaction. One weak spot of log
> > compaction
> > > > is
> > > > > > for
> > > > > > > > the
> > > > > > > > > >> > > consumer
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > > deal with deletes. When a key is deleted, it's
> > > > retained
> > > > > > as a
> > > > > > > > > >> > tombstone
> > > > > > > > > >> > > > > first and then physically removed. If a client
> > > misses
> > > > > the
> > > > > > > > > >> tombstone
> > > > > > > > > >> > > > > (because it's physically removed), it may not be
> > > able
> > > > to
> > > > > > > > update
> > > > > > > > > >> its
> > > > > > > > > >> > > > > metadata properly. The way we solve this in
> Kafka
> > is
> > > > > based
> > > > > > > on
> > > > > > > > a
> > > > > > > > > >> > > > > configuration (log.cleaner.delete.retention.ms)
> > and
> > > > we
> > > > > > > > expect a
> > > > > > > > > >> > > consumer
> > > > > > > > > >> > > > > having seen an old key to finish reading the
> > > deletion
> > > > > > > > tombstone
> > > > > > > > > >> > within
> > > > > > > > > >> > > > that
> > > > > > > > > >> > > > > time. There is no strong guarantee for that
> since
> > a
> > > > > broker
> > > > > > > > could
> > > > > > > > > >> be
> > > > > > > > > >> > > down
> > > > > > > > > >> > > > > for a long time. It would be better if we can
> > have a
> > > > > more
> > > > > > > > > reliable
> > > > > > > > > >> > way
> > > > > > > > > >> > > of
> > > > > > > > > >> > > > > dealing with deletes.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > We propose to capture this in the
> "FirstDirtyOffset"
> > > > field
> > > > > > of
> > > > > > > > the
> > > > > > > > > >> > quorum
> > > > > > > > > >> > > > record fetch response: the offset is the maximum
> > > offset
> > > > > that
> > > > > > > log
> > > > > > > > > >> > > compaction
> > > > > > > > > >> > > > has reached up to. If the follower has fetched
> > beyond
> > > > this
> > > > > > > > offset
> > > > > > > > > it
> > > > > > > > > >> > > means
> > > > > > > > > >> > > > itself is safe hence it has seen all records up to
> > > that
> > > > > > > offset.
> > > > > > > > On
> > > > > > > > > >> > > getting
> > > > > > > > > >> > > > the response, the follower can then decide if its
> > end
> > > > > offset
> > > > > > > > > >> actually
> > > > > > > > > >> > > below
> > > > > > > > > >> > > > that dirty offset (and hence may miss some
> > > tombstones).
> > > > If
> > > > > > > > that's
> > > > > > > > > >> the
> > > > > > > > > >> > > case:
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log
> from
> > > the
> > > > > very
> > > > > > > > > >> beginning
> > > > > > > > > >> > to
> > > > > > > > > >> > > > catch up.
> > > > > > > > > >> > > > 2) During that time, it would refrain itself from
> > > > > answering
> > > > > > > > > >> > > MetadataRequest
> > > > > > > > > >> > > > from any clients.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > I am not sure that the "FirstDirtyOffset" field
> fully
> > > > > > addresses
> > > > > > > > the
> > > > > > > > > >> > issue.
> > > > > > > > > >> > > Currently, the deletion tombstone is not removed
> > > > immediately
> > > > > > > > after a
> > > > > > > > > >> > round
> > > > > > > > > >> > > of cleaning. It's removed after a delay in a
> > subsequent
> > > > > round
> > > > > > of
> > > > > > > > > >> > cleaning.
> > > > > > > > > >> > > Consider an example where a key insertion is at
> offset
> > > 200
> > > > > > and a
> > > > > > > > > >> deletion
> > > > > > > > > >> > > tombstone of the key is at 400. Initially,
> > > > FirstDirtyOffset
> > > > > is
> > > > > > > at
> > > > > > > > > >> 300. A
> > > > > > > > > >> > > follower/observer fetches from offset 0  and fetches
> > the
> > > > key
> > > > > > at
> > > > > > > > > offset
> > > > > > > > > >> > 200.
> > > > > > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is
> > at
> > > > 500
> > > > > > and
> > > > > > > > the
> > > > > > > > > >> > > tombstone at 400 is physically removed. The
> > > > > follower/observer
> > > > > > > > > >> continues
> > > > > > > > > >> > the
> > > > > > > > > >> > > fetch, but misses offset 400. It catches all the way
> > to
> > > > > > > > > >> FirstDirtyOffset
> > > > > > > > > >> > > and declares its metadata as ready. However, its
> > > metadata
> > > > > > could
> > > > > > > be
> > > > > > > > > >> stale
> > > > > > > > > >> > > since it actually misses the deletion of the key.
> > > > > > > > > >> > >
> > > > > > > > > >> > > Yeah good question, I should have put more details
> in
> > my
> > > > > > > > explanation
> > > > > > > > > >> :)
> > > > > > > > > >> >
> > > > > > > > > >> > The idea is that we will adjust the log compaction for
> > > this
> > > > > raft
> > > > > > > > based
> > > > > > > > > >> > metadata log: before more details to be explained,
> since
> > > we
> > > > > have
> > > > > > > two
> > > > > > > > > >> types
> > > > > > > > > >> > of "watermarks" here, whereas in Kafka the watermark
> > > > indicates
> > > > > > > where
> > > > > > > > > >> every
> > > > > > > > > >> > replica have replicated up to and in Raft the
> watermark
> > > > > > indicates
> > > > > > > > > where
> > > > > > > > > >> the
> > > > > > > > > >> > majority of replicas (here only indicating voters of
> the
> > > > > quorum,
> > > > > > > not
> > > > > > > > > >> > counting observers) have replicated up to, let's call
> > them
> > > > > Kafka
> > > > > > > > > >> watermark
> > > > > > > > > >> > and Raft watermark. For this special log, we would
> > > maintain
> > > > > both
> > > > > > > > > >> > watermarks.
> > > > > > > > > >> >
> > > > > > > > > >> > When log compacting on the leader, we would only
> compact
> > > up
> > > > to
> > > > > > the
> > > > > > > > > Kafka
> > > > > > > > > >> > watermark, i.e. if there is at least one voter who
> have
> > > not
> > > > > > > > replicated
> > > > > > > > > >> an
> > > > > > > > > >> > entry, it would not be compacted. The "dirty-offset"
> is
> > > the
> > > > > > offset
> > > > > > > > > that
> > > > > > > > > >> > we've compacted up to and is communicated to other
> > voters,
> > > > and
> > > > > > the
> > > > > > > > > other
> > > > > > > > > >> > voters would also compact up to this value --- i.e.
> the
> > > > > > difference
> > > > > > > > > here
> > > > > > > > > >> is
> > > > > > > > > >> > that instead of letting each replica doing log
> > compaction
> > > > > > > > > independently,
> > > > > > > > > >> > we'll have the leader to decide upon which offset to
> > > compact
> > > > > to,
> > > > > > > and
> > > > > > > > > >> > propagate this value to others to follow, in a more
> > > > > coordinated
> > > > > > > > > manner.
> > > > > > > > > >> > Also note when there are new voters joining the quorum
> > who
> > > > has
> > > > > > not
> > > > > > > > > >> > replicated up to the dirty-offset, of because of other
> > > > issues
> > > > > > they
> > > > > > > > > >> > truncated their logs to below the dirty-offset, they'd
> > > have
> > > > to
> > > > > > > > > >> re-bootstrap
> > > > > > > > > >> > from the beginning, and during this period of time the
> > > > leader
> > > > > > > > learned
> > > > > > > > > >> about
> > > > > > > > > >> > this lagging voter would not advance the watermark
> (also
> > > it
> > > > > > would
> > > > > > > > not
> > > > > > > > > >> > decrement it), and hence not compacting either, until
> > the
> > > > > > voter(s)
> > > > > > > > has
> > > > > > > > > >> > caught up to that dirty-offset.
> > > > > > > > > >> >
> > > > > > > > > >> > So back to your example above, before the bootstrap
> > voter
> > > > gets
> > > > > > to
> > > > > > > > 300
> > > > > > > > > no
> > > > > > > > > >> > log compaction would happen on the leader; and until
> > later
> > > > > when
> > > > > > > the
> > > > > > > > > >> voter
> > > > > > > > > >> > have got to beyond 400 and hence replicated that
> > > tombstone,
> > > > > the
> > > > > > > log
> > > > > > > > > >> > compaction would possibly get to that tombstone and
> > remove
> > > > it.
> > > > > > Say
> > > > > > > > > >> later it
> > > > > > > > > >> > the leader's log compaction reaches 500, it can send
> > this
> > > > back
> > > > > > to
> > > > > > > > the
> > > > > > > > > >> voter
> > > > > > > > > >> > who can then also compact locally up to 500.
> > > > > > > > > >> >
> > > > > > > > > >> >
> > > > > > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we
> > > need
> > > > > the
> > > > > > > > epoch
> > > > > > > > > >> > > > > corresponding to VotedId? Over time, the same
> > broker
> > > > Id
> > > > > > > could
> > > > > > > > be
> > > > > > > > > >> > voted
> > > > > > > > > >> > > in
> > > > > > > > > >> > > > > different generations with different epoch.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > > > > > "LeaderEpoch"
> > > > > > > > > >> field
> > > > > > > > > >> > in
> > > > > > > > > >> > > > that file is corresponding to the "latest known
> > leader
> > > > > > epoch",
> > > > > > > > not
> > > > > > > > > >> the
> > > > > > > > > >> > > > "current leader epoch". For example, if the
> current
> > > > epoch
> > > > > is
> > > > > > > N,
> > > > > > > > > and
> > > > > > > > > >> > then
> > > > > > > > > >> > > a
> > > > > > > > > >> > > > vote-request with epoch N+1 is received and the
> > voter
> > > > > > granted
> > > > > > > > the
> > > > > > > > > >> vote
> > > > > > > > > >> > > for
> > > > > > > > > >> > > > it, then it means for this voter it knows the
> > "latest
> > > > > epoch"
> > > > > > > is
> > > > > > > > N
> > > > > > > > > +
> > > > > > > > > >> 1
> > > > > > > > > >> > > > although it is unknown if that sending candidate
> > will
> > > > > indeed
> > > > > > > > > become
> > > > > > > > > >> the
> > > > > > > > > >> > > new
> > > > > > > > > >> > > > leader (which would only be notified via
> > begin-quorum
> > > > > > > request).
> > > > > > > > > >> > However,
> > > > > > > > > >> > > > when persisting the quorum state, we would encode
> > > > > > leader-epoch
> > > > > > > > to
> > > > > > > > > >> N+1,
> > > > > > > > > >> > > > while the leaderId to be the older leader.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > But now thinking about this a bit more, I feel we
> > > should
> > > > > use
> > > > > > > two
> > > > > > > > > >> > separate
> > > > > > > > > >> > > > epochs, one for the "lates known" and one for the
> > > > > "current"
> > > > > > to
> > > > > > > > > pair
> > > > > > > > > >> > with
> > > > > > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch
> > > before
> > > > > the
> > > > > > > new
> > > > > > > > > >> leader
> > > > > > > > > >> > > is actually elected, right.
> > > > > > > > > >> > >
> > > > > > > > > >> > >
> > > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > > > > FetchQuorumRecords
> > > > > > > API
> > > > > > > > to
> > > > > > > > > >> > > indicate
> > > > > > > > > >> > > > > that the follower has fetched from an invalid
> > offset
> > > > and
> > > > > > > > should
> > > > > > > > > >> > > truncate
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > > the offset/epoch indicated in the response."
> > > Observers
> > > > > > can't
> > > > > > > > > >> truncate
> > > > > > > > > >> > > > their
> > > > > > > > > >> > > > > logs. What should they do with
> > OFFSET_OUT_OF_RANGE?
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > I'm not sure if I understand your question?
> > Observers
> > > > > should
> > > > > > > > still
> > > > > > > > > >> be
> > > > > > > > > >> > > able
> > > > > > > > > >> > > > to truncate their logs as well.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > Hmm, I thought only the quorum nodes have local logs
> > and
> > > > > > > observers
> > > > > > > > > >> don't?
> > > > > > > > > >> > >
> > > > > > > > > >> > > > 107. "The leader will continue sending
> > > BeginQuorumEpoch
> > > > to
> > > > > > > each
> > > > > > > > > >> known
> > > > > > > > > >> > > > voter
> > > > > > > > > >> > > > > until it has received its endorsement." If a
> voter
> > > is
> > > > > down
> > > > > > > > for a
> > > > > > > > > >> long
> > > > > > > > > >> > > > time,
> > > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add
> unnecessary
> > > > > > overhead.
> > > > > > > > > >> > Similarly,
> > > > > > > > > >> > > > if a
> > > > > > > > > >> > > > > follower stops sending FetchQuorumRecords, does
> > the
> > > > > leader
> > > > > > > > keep
> > > > > > > > > >> > sending
> > > > > > > > > >> > > > > BeginQuorumEpoch?
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point.
> > The
> > > > > > > > > >> > begin-quorum-epoch
> > > > > > > > > >> > > > request is for voters to quickly get the new
> leader
> > > > > > > information;
> > > > > > > > > >> > however
> > > > > > > > > >> > > > even if they do not get them they can still
> > eventually
> > > > > learn
> > > > > > > > about
> > > > > > > > > >> that
> > > > > > > > > >> > > > from others via gossiping FindQuorum. I think we
> can
> > > > > adjust
> > > > > > > the
> > > > > > > > > >> logic
> > > > > > > > > >> > to
> > > > > > > > > >> > > > e.g. exponential back-off or with a limited
> > > num.retries.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Regarding FetchQuorumRecords: if the follower
> sends
> > > > > > > > > >> FetchQuorumRecords
> > > > > > > > > >> > > > already, it means that follower already knows that
> > the
> > > > > > broker
> > > > > > > is
> > > > > > > > > the
> > > > > > > > > >> > > > leader, and hence we can stop retrying
> > > BeginQuorumEpoch;
> > > > > > > however
> > > > > > > > > it
> > > > > > > > > >> is
> > > > > > > > > >> > > > possible that after a follower sends
> > > FetchQuorumRecords
> > > > > > > already,
> > > > > > > > > >> > suddenly
> > > > > > > > > >> > > > it stops send it (possibly because it learned
> about
> > a
> > > > > higher
> > > > > > > > epoch
> > > > > > > > > >> > > leader),
> > > > > > > > > >> > > > and hence this broker may be a "zombie" leader and
> > we
> > > > > > propose
> > > > > > > to
> > > > > > > > > use
> > > > > > > > > >> > the
> > > > > > > > > >> > > > fetch.timeout to let the leader to try to verify
> if
> > it
> > > > has
> > > > > > > > already
> > > > > > > > > >> been
> > > > > > > > > >> > > > stale.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > It just seems that we should handle these two cases
> > in a
> > > > > > > > consistent
> > > > > > > > > >> way?
> > > > > > > > > >> > >
> > > > > > > > > >> > > Yes I agree, on the leader's side, the
> > > FetchQuorumRecords
> > > > > > from a
> > > > > > > > > >> follower
> > > > > > > > > >> > could mean that we no longer needs to send
> > > BeginQuorumEpoch
> > > > > > > anymore
> > > > > > > > > ---
> > > > > > > > > >> and
> > > > > > > > > >> > it is already part of our current implementations in
> > > > > > > > > >> >
> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > > > > > >> >
> > > > > > > > > >> >
> > > > > > > > > >> > > Thanks,
> > > > > > > > > >> > >
> > > > > > > > > >> > > Jun
> > > > > > > > > >> > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Jun
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > > > > > wangguoz@gmail.com
> > > > > > > > > >> >
> > > > > > > > > >> > > > wrote:
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > > Hello Leonard,
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Thanks for your comments, I'm relying in line
> > > below:
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard)
> > Ge
> > > <
> > > > > > > > > >> > wge@confluent.io>
> > > > > > > > > >> > > > > > wrote:
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > > Hi Kafka developers,
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > It's great to see this proposal and it took
> me
> > > > some
> > > > > > time
> > > > > > > > to
> > > > > > > > > >> > finish
> > > > > > > > > >> > > > > > reading
> > > > > > > > > >> > > > > > > it.
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > And I have the following questions about the
> > > > > Proposal:
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > >    - How do we plan to test this design to
> > > ensure
> > > > > its
> > > > > > > > > >> > correctness?
> > > > > > > > > >> > > Or
> > > > > > > > > >> > > > > > more
> > > > > > > > > >> > > > > > >    broadly, how do we ensure that our new
> > ‘pull’
> > > > > based
> > > > > > > > model
> > > > > > > > > >> is
> > > > > > > > > >> > > > > > functional
> > > > > > > > > >> > > > > > > and
> > > > > > > > > >> > > > > > >    correct given that it is different from
> the
> > > > > > original
> > > > > > > > RAFT
> > > > > > > > > >> > > > > > implementation
> > > > > > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > We have two planned verifications on the
> > > correctness
> > > > > and
> > > > > > > > > >> liveness
> > > > > > > > > >> > of
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > > > > > >> > > > > >
> > > https://github.com/guozhangwang/kafka-specification
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >> >
> > > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >    - Have we considered any sensible defaults
> > for
> > > > the
> > > > > > > > > >> > configuration,
> > > > > > > > > >> > > > i.e.
> > > > > > > > > >> > > > > > >    all the election timeout, fetch time out,
> > > etc.?
> > > > > Or
> > > > > > we
> > > > > > > > > want
> > > > > > > > > >> to
> > > > > > > > > >> > > > leave
> > > > > > > > > >> > > > > > > this to
> > > > > > > > > >> > > > > > >    a later stage when we do the performance
> > > > testing,
> > > > > > > etc.
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > This is a good question, the reason we did not
> > set
> > > > any
> > > > > > > > default
> > > > > > > > > >> > values
> > > > > > > > > >> > > > for
> > > > > > > > > >> > > > > > the timeout configurations is that we think it
> > may
> > > > > take
> > > > > > > some
> > > > > > > > > >> > > > benchmarking
> > > > > > > > > >> > > > > > experiments to get these defaults right. Some
> > > > > high-level
> > > > > > > > > >> principles
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > > > consider: 1) the fetch.timeout should be
> around
> > > the
> > > > > same
> > > > > > > > scale
> > > > > > > > > >> with
> > > > > > > > > >> > > zk
> > > > > > > > > >> > > > > > session timeout, which is now 18 seconds by
> > > default
> > > > --
> > > > > > in
> > > > > > > > > >> practice
> > > > > > > > > >> > > > we've
> > > > > > > > > >> > > > > > seen unstable networks having more than 10
> secs
> > of
> > > > > > > transient
> > > > > > > > > >> > > > > connectivity,
> > > > > > > > > >> > > > > > 2) the election.timeout, however, should be
> > > smaller
> > > > > than
> > > > > > > the
> > > > > > > > > >> fetch
> > > > > > > > > >> > > > > timeout
> > > > > > > > > >> > > > > > as is also suggested as a practical
> optimization
> > > in
> > > > > > > > > literature:
> > > > > > > > > >> > > > > >
> > > > > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Some more discussions can be found here:
> > > > > > > > > >> > > > > >
> > > > > > > > >
> > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > >    - Have we considered piggybacking
> > > > > > `BeginQuorumEpoch`
> > > > > > > > with
> > > > > > > > > >> the
> > > > > > > > > >> > `
> > > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing
> > > > something
> > > > > > > > obvious
> > > > > > > > > >> but
> > > > > > > > > >> > I
> > > > > > > > > >> > > am
> > > > > > > > > >> > > > > > just
> > > > > > > > > >> > > > > > >    wondering why don’t we just use the
> > > > `FindQuorum`
> > > > > > and
> > > > > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch`
> API?
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from
> > leader
> > > > ->
> > > > > > > other
> > > > > > > > > >> voter
> > > > > > > > > >> > > > > > followers, while FindQuorum / Fetch are sent
> > from
> > > > > > follower
> > > > > > > > to
> > > > > > > > > >> > leader.
> > > > > > > > > >> > > > > > Arguably one can eventually realize the new
> > leader
> > > > and
> > > > > > > epoch
> > > > > > > > > via
> > > > > > > > > >> > > > > gossiping
> > > > > > > > > >> > > > > > FindQuorum, but that could in practice
> require a
> > > > long
> > > > > > > delay.
> > > > > > > > > >> > Having a
> > > > > > > > > >> > > > > > leader -> other voters request helps the new
> > > leader
> > > > > > epoch
> > > > > > > to
> > > > > > > > > be
> > > > > > > > > >> > > > > propagated
> > > > > > > > > >> > > > > > faster under a pull model.
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > >    - And about the `FetchQuorumRecords`
> > response
> > > > > > schema,
> > > > > > > > in
> > > > > > > > > >> the
> > > > > > > > > >> > > > > `Records`
> > > > > > > > > >> > > > > > >    field of the response, is it just one
> > record
> > > or
> > > > > all
> > > > > > > the
> > > > > > > > > >> > records
> > > > > > > > > >> > > > > > starting
> > > > > > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> > > > > efficient
> > > > > > > if
> > > > > > > > we
> > > > > > > > > >> sent
> > > > > > > > > >> > > all
> > > > > > > > > >> > > > > the
> > > > > > > > > >> > > > > > >    records during the bootstrapping of the
> > > > brokers.
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is
> just
> > > the
> > > > > > > > starting
> > > > > > > > > >> > offset
> > > > > > > > > >> > > of
> > > > > > > > > >> > > > > the
> > > > > > > > > >> > > > > > batch of records.
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > >    - Regarding the disruptive broker issues,
> > > does
> > > > > our
> > > > > > > pull
> > > > > > > > > >> based
> > > > > > > > > >> > > > model
> > > > > > > > > >> > > > > > >    suffer from it? If so, have we considered
> > the
> > > > > > > Pre-Vote
> > > > > > > > > >> stage?
> > > > > > > > > >> > If
> > > > > > > > > >> > > > > not,
> > > > > > > > > >> > > > > > > why?
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > The disruptive broker is stated in the
> original
> > > Raft
> > > > > > paper
> > > > > > > > > >> which is
> > > > > > > > > >> > > the
> > > > > > > > > >> > > > > > result of the push model design. Our analysis
> > > showed
> > > > > > that
> > > > > > > > with
> > > > > > > > > >> the
> > > > > > > > > >> > > pull
> > > > > > > > > >> > > > > > model it is no longer an issue.
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > > Thanks a lot for putting this up, and I hope
> > > that
> > > > my
> > > > > > > > > questions
> > > > > > > > > >> > can
> > > > > > > > > >> > > be
> > > > > > > > > >> > > > > of
> > > > > > > > > >> > > > > > > some value to make this KIP better.
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > Hope to hear from you soon!
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > Best wishes,
> > > > > > > > > >> > > > > > > Leonard
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin
> McCabe <
> > > > > > > > > >> cmccabe@apache.org
> > > > > > > > > >> > >
> > > > > > > > > >> > > > > wrote:
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > > Hi Jason,
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > It's amazing to see this coming together
> :)
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > I haven't had a chance to read in detail,
> > but
> > > I
> > > > > read
> > > > > > > the
> > > > > > > > > >> > outline
> > > > > > > > > >> > > > and
> > > > > > > > > >> > > > > a
> > > > > > > > > >> > > > > > > few
> > > > > > > > > >> > > > > > > > things jumped out at me.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > First, for every epoch that is 32 bits
> > rather
> > > > than
> > > > > > > 64, I
> > > > > > > > > >> sort
> > > > > > > > > >> > of
> > > > > > > > > >> > > > > wonder
> > > > > > > > > >> > > > > > > if
> > > > > > > > > >> > > > > > > > that's a good long-term choice.  I keep
> > > reading
> > > > > > about
> > > > > > > > > stuff
> > > > > > > > > >> > like
> > > > > > > > > >> > > > > this:
> > > > > > > > > >> > > > > > > >
> > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > > > > .
> > > > > > > > > >> > > Obviously,
> > > > > > > > > >> > > > > > that
> > > > > > > > > >> > > > > > > > JIRA is about zxid, which increments much
> > > faster
> > > > > > than
> > > > > > > we
> > > > > > > > > >> expect
> > > > > > > > > >> > > > these
> > > > > > > > > >> > > > > > > > leader epochs to, but it would still be
> good
> > > to
> > > > > see
> > > > > > > some
> > > > > > > > > >> rough
> > > > > > > > > >> > > > > > > calculations
> > > > > > > > > >> > > > > > > > about how long 32 bits (or really, 31
> bits)
> > > will
> > > > > > last
> > > > > > > us
> > > > > > > > > in
> > > > > > > > > >> the
> > > > > > > > > >> > > > cases
> > > > > > > > > >> > > > > > > where
> > > > > > > > > >> > > > > > > > we're using it, and what the space savings
> > > we're
> > > > > > > getting
> > > > > > > > > >> really
> > > > > > > > > >> > > is.
> > > > > > > > > >> > > > > It
> > > > > > > > > >> > > > > > > > seems like in most cases the tradeoff may
> > not
> > > be
> > > > > > worth
> > > > > > > > it?
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > Another thing I've been thinking about is
> > how
> > > we
> > > > > do
> > > > > > > > > >> > > > bootstrapping.  I
> > > > > > > > > >> > > > > > > > would prefer to be in a world where
> > > formatting a
> > > > > new
> > > > > > > > Kafka
> > > > > > > > > >> node
> > > > > > > > > >> > > > was a
> > > > > > > > > >> > > > > > > first
> > > > > > > > > >> > > > > > > > class operation explicitly initiated by
> the
> > > > admin,
> > > > > > > > rather
> > > > > > > > > >> than
> > > > > > > > > >> > > > > > something
> > > > > > > > > >> > > > > > > > that happened implicitly when you started
> up
> > > the
> > > > > > > broker
> > > > > > > > > and
> > > > > > > > > >> > > things
> > > > > > > > > >> > > > > > > "looked
> > > > > > > > > >> > > > > > > > blank."
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > The first problem is that things can "look
> > > > blank"
> > > > > > > > > >> accidentally
> > > > > > > > > >> > if
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > > storage system is having a bad day.
> Clearly
> > > in
> > > > > the
> > > > > > > > > non-Raft
> > > > > > > > > >> > > world,
> > > > > > > > > >> > > > > > this
> > > > > > > > > >> > > > > > > > leads to data loss if the broker that is
> > > > > (re)started
> > > > > > > > this
> > > > > > > > > >> way
> > > > > > > > > >> > was
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > > leader for some partitions.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > The second problem is that we have a bit
> of
> > a
> > > > > > chicken
> > > > > > > > and
> > > > > > > > > >> egg
> > > > > > > > > >> > > > problem
> > > > > > > > > >> > > > > > > with
> > > > > > > > > >> > > > > > > > certain configuration keys.  For example,
> > > maybe
> > > > > you
> > > > > > > want
> > > > > > > > > to
> > > > > > > > > >> > > > configure
> > > > > > > > > >> > > > > > > some
> > > > > > > > > >> > > > > > > > connection security settings in your
> > cluster,
> > > > but
> > > > > > you
> > > > > > > > > don't
> > > > > > > > > >> > want
> > > > > > > > > >> > > > them
> > > > > > > > > >> > > > > > to
> > > > > > > > > >> > > > > > > > ever be stored in a plaintext config file.
> > > (For
> > > > > > > > example,
> > > > > > > > > >> SCRAM
> > > > > > > > > >> > > > > > > passwords,
> > > > > > > > > >> > > > > > > > etc.)  You could use a broker API to set
> the
> > > > > > > > > configuration,
> > > > > > > > > >> but
> > > > > > > > > >> > > > that
> > > > > > > > > >> > > > > > > brings
> > > > > > > > > >> > > > > > > > up the chicken and egg problem.  The
> broker
> > > > needs
> > > > > to
> > > > > > > be
> > > > > > > > > >> > > configured
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > > > > know
> > > > > > > > > >> > > > > > > > how to talk to you, but you need to
> > configure
> > > it
> > > > > > > before
> > > > > > > > > you
> > > > > > > > > >> can
> > > > > > > > > >> > > > talk
> > > > > > > > > >> > > > > to
> > > > > > > > > >> > > > > > > > it.  Using an external secret manager like
> > > Vault
> > > > > is
> > > > > > > one
> > > > > > > > > way
> > > > > > > > > >> to
> > > > > > > > > >> > > > solve
> > > > > > > > > >> > > > > > > this,
> > > > > > > > > >> > > > > > > > but not everyone uses an external secret
> > > > manager.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > quorum.voters seems like a similar
> > > configuration
> > > > > > key.
> > > > > > > > In
> > > > > > > > > >> the
> > > > > > > > > >> > > > current
> > > > > > > > > >> > > > > > > KIP,
> > > > > > > > > >> > > > > > > > this is only read if there is no other
> > > > > configuration
> > > > > > > > > >> specifying
> > > > > > > > > >> > > the
> > > > > > > > > >> > > > > > > quorum
> > > > > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs
> command,
> > we
> > > > > > > wouldn't
> > > > > > > > > need
> > > > > > > > > >> > this
> > > > > > > > > >> > > > key
> > > > > > > > > >> > > > > > > > because we could assume that there was
> > always
> > > > > quorum
> > > > > > > > > >> > information
> > > > > > > > > >> > > > > stored
> > > > > > > > > >> > > > > > > > locally.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > best,
> > > > > > > > > >> > > > > > > > Colin
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason
> > > Gustafson
> > > > > > wrote:
> > > > > > > > > >> > > > > > > > > Hi All,
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > I'd like to start a discussion on
> KIP-595:
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >> >
> > > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > > > > >> > > > > > > > .
> > > > > > > > > >> > > > > > > > > This proposal specifies a Raft protocol
> to
> > > > > > > ultimately
> > > > > > > > > >> replace
> > > > > > > > > >> > > > > > Zookeeper
> > > > > > > > > >> > > > > > > > > as
> > > > > > > > > >> > > > > > > > > documented in KIP-500. Please take a
> look
> > > and
> > > > > > share
> > > > > > > > your
> > > > > > > > > >> > > > thoughts.
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > A few minor notes to set the stage a
> > little
> > > > bit:
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > - This KIP does not specify the
> structure
> > of
> > > > the
> > > > > > > > > messages
> > > > > > > > > >> > used
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > > > > > represent
> > > > > > > > > >> > > > > > > > > metadata in Kafka, nor does it specify
> the
> > > > > > internal
> > > > > > > > API
> > > > > > > > > >> that
> > > > > > > > > >> > > will
> > > > > > > > > >> > > > > be
> > > > > > > > > >> > > > > > > used
> > > > > > > > > >> > > > > > > > > by the controller. Expect these to come
> in
> > > > later
> > > > > > > > > >> proposals.
> > > > > > > > > >> > > Here
> > > > > > > > > >> > > > we
> > > > > > > > > >> > > > > > are
> > > > > > > > > >> > > > > > > > > primarily concerned with the replication
> > > > > protocol
> > > > > > > and
> > > > > > > > > >> basic
> > > > > > > > > >> > > > > > operational
> > > > > > > > > >> > > > > > > > > mechanics.
> > > > > > > > > >> > > > > > > > > - We expect many details to change as we
> > get
> > > > > > closer
> > > > > > > to
> > > > > > > > > >> > > > integration
> > > > > > > > > >> > > > > > with
> > > > > > > > > >> > > > > > > > > the controller. Any changes we make will
> > be
> > > > made
> > > > > > > > either
> > > > > > > > > as
> > > > > > > > > >> > > > > amendments
> > > > > > > > > >> > > > > > > to
> > > > > > > > > >> > > > > > > > > this KIP or, in the case of larger
> > changes,
> > > as
> > > > > new
> > > > > > > > > >> proposals.
> > > > > > > > > >> > > > > > > > > - We have a prototype implementation
> > which I
> > > > > will
> > > > > > > put
> > > > > > > > > >> online
> > > > > > > > > >> > > > within
> > > > > > > > > >> > > > > > the
> > > > > > > > > >> > > > > > > > > next week which may help in
> understanding
> > > some
> > > > > > > > details.
> > > > > > > > > It
> > > > > > > > > >> > has
> > > > > > > > > >> > > > > > > diverged a
> > > > > > > > > >> > > > > > > > > little bit from our proposal, so I am
> > > taking a
> > > > > > > little
> > > > > > > > > >> time to
> > > > > > > > > >> > > > bring
> > > > > > > > > >> > > > > > it
> > > > > > > > > >> > > > > > > in
> > > > > > > > > >> > > > > > > > > line. I'll post an update to this thread
> > > when
> > > > it
> > > > > > is
> > > > > > > > > >> available
> > > > > > > > > >> > > for
> > > > > > > > > >> > > > > > > review.
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > Finally, I want to mention that this
> > > proposal
> > > > > was
> > > > > > > > > drafted
> > > > > > > > > >> by
> > > > > > > > > >> > > > > myself,
> > > > > > > > > >> > > > > > > > Boyang
> > > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > >> > > > > > > > > Jason
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > --
> > > > > > > > > >> > > > > > > Leonard Ge
> > > > > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > --
> > > > > > > > > >> > > > > > -- Guozhang
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > --
> > > > > > > > > >> > > > -- Guozhang
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >> >
> > > > > > > > > >> >
> > > > > > > > > >> > --
> > > > > > > > > >> > -- Guozhang
> > > > > > > > > >> >
> > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > -- Guozhang
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Unmesh Joshi <un...@gmail.com>.
Hi,

In the FetchRequest Handling, how to make sure we handle scenarios where
the leader might have been disconnected from the cluster, but doesn't know
yet?
As discussed in the Raft Thesis section 6.4, the linearizable semantics of
read requests is implemented in LogCabin by sending heartbeat to followers
and waiting till the heartbeats are successful to make sure that the leader
is still the leader.
I think for the controller quorum to make sure none of the consumers get
stale data, it's important to have linearizable semantics? In the pull
based model, the leader will need to wait for heartbeats from the followers
before returning each fetch request from the consumer then? Or do we need
to introduce some other request?
(Zookeeper does not have linearizable semantics for read requests, but as
of now all the kafka interactions are through writes and watches).

Thanks,
Unmesh

On Fri, Jul 24, 2020 at 11:36 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Jason,
>
> Thanks for the reply.
>
> 101. Sounds good. Regarding clusterId, I am not sure storing it in the
> metadata log is enough. For example, the vote request includes clusterId.
> So, no one can vote until they know the clusterId. Also, it would be useful
> to support the case when a voter completely loses its disk and needs to
> recover.
>
> 210. There is no longer a FindQuorum request. When a follower restarts, how
> does it discover the leader? Is that based on DescribeQuorum? It would be
> useful to document this.
>
> Jun
>
> On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi Jun,
> >
> > Thanks for the questions.
> >
> > 101. I am treating some of the bootstrapping problems as out of the scope
> > of this KIP. I am working on a separate proposal which addresses
> > bootstrapping security credentials specifically. Here is a rough sketch
> of
> > how I am seeing it:
> >
> > 1. Dynamic broker configurations including encrypted passwords will be
> > persisted in the metadata log and cached in the broker's
> `meta.properties`
> > file.
> > 2. We will provide a tool which allows users to directly override the
> > values in `meta.properties` without requiring access to the quorum. This
> > can be used to bootstrap the credentials of the voter set itself before
> the
> > cluster has been started.
> > 3. Some dynamic config changes will only be allowed when a broker is
> > online. For example, changing a truststore password dynamically would
> > prevent that broker from being able to start if it were offline when the
> > change was made.
> > 4. I am still thinking a little bit about SCRAM credentials, but most
> > likely they will be handled with an approach similar to
> `meta.properties`.
> >
> > 101.3 As for the question about `clusterId`, I think the way we would do
> > this is to have the first elected leader generate a UUID and write it to
> > the metadata log. Let me add some detail to the proposal about this.
> >
> > A few additional answers below:
> >
> > 203. Yes, that is correct.
> >
> > 204. That is a good question. What happens in this case is that all
> voters
> > advance their epoch to the one designated by the candidate even if they
> > reject its vote request. Assuming the candidate fails to be elected, the
> > election will be retried until a leader emerges.
> >
> > 205. I had some discussion with Colin offline about this problem. I think
> > the answer should be "yes," but it probably needs a little more thought.
> > Handling JBOD failures is tricky. For an observer, we can replicate the
> > metadata log from scratch safely in a new log dir. But if the log dir of
> a
> > voter fails, I do not think it is generally safe to start from an empty
> > state.
> >
> > 206. Yes, that is discussed in KIP-631 I believe.
> >
> > 207. Good suggestion. I will work on this.
> >
> >
> > Thanks,
> > Jason
> >
> >
> >
> >
> >
> > On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Jason,
> > >
> > > Thanks for the updated KIP. Looks good overall. A few more comments
> > below.
> > >
> > > 101. I still don't see a section on bootstrapping related issues. It
> > would
> > > be useful to document if/how the following is supported.
> > > 101.1 Currently, we support auto broker id generation. Is this
> supported
> > > for bootstrap brokers?
> > > 101.2 As Colin mentioned, sometimes we may need to load the security
> > > credentials to be broker before it can be connected to. Could you
> > provide a
> > > bit more detail on how this will work?
> > > 101.3 Currently, we use ZK to generate clusterId on a new cluster. With
> > > Raft, how does every broker generate the same clusterId in a
> distributed
> > > way?
> > >
> > > 200. It would be useful to document if the various special offsets (log
> > > start offset, recovery point, HWM, etc) for the Raft log are stored in
> > the
> > > same existing checkpoint files or not.
> > > 200.1 Since the Raft log flushes every append, does that allow us to
> > > recover from a recovery point within the active segment or do we still
> > need
> > > to scan the full segment including the recovery point? The former can
> be
> > > tricky since multiple records can fall into the same disk page and a
> > > subsequent flush may corrupt a page with previously flushed records.
> > >
> > > 201. Configurations.
> > > 201.1 How do the Raft brokers get security related configs for inter
> > broker
> > > communication? Is that based on the existing
> > > inter.broker.security.protocol?
> > > 201.2 We have quorum.retry.backoff.max.ms and quorum.retry.backoff.ms,
> > but
> > > only quorum.election.backoff.max.ms. This seems a bit inconsistent.
> > >
> > > 202. Metrics:
> > > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs,
> OutboundQueueTimeMs:
> > > Are those the same as existing totalTime, requestQueueTime, localTime,
> > > responseQueueTime? Could we reuse the existing ones with the tag
> > > request=[request-type]?
> > > 202.2. Could you explain what InboundChannelSize and
> OutboundChannelSize
> > > are?
> > > 202.3 ElectionLatencyMax/Avg: It seems that both should be windowed?
> > >
> > > 203. Quorum State: I assume that LeaderId will be kept consistently
> with
> > > LeaderEpoch. For example, if a follower transitions to candidate and
> > bumps
> > > up LeaderEpoch, it will set leaderId to -1 and persist both in the
> Quorum
> > > state file. Is that correct?
> > >
> > > 204. I was thinking about a corner case when a Raft broker is
> partitioned
> > > off. This broker will then be in a continuous loop of bumping up the
> > leader
> > > epoch, but failing to get enough votes. When the partitioning is
> removed,
> > > this broker's high leader epoch will force a leader election. I assume
> > > other Raft brokers can immediately advance their leader epoch passing
> the
> > > already bumped epoch such that leader election won't be delayed. Is
> that
> > > right?
> > >
> > > 205. In a JBOD setting, could we use the existing tool to move the Raft
> > log
> > > from one disk to another?
> > >
> > > 206. The KIP doesn't mention the local metadata store derived from the
> > Raft
> > > log. Will that be covered in a separate KIP?
> > >
> > > 207. Since this is a critical component. Could we add a section on the
> > > testing plan for correctness?
> > >
> > > 208. Performance. Do we plan to do group commit (e.g. buffer pending
> > > appends during a flush and then flush all accumulated pending records
> > > together in the next flush) for better throughput?
> > >
> > > 209. "the leader can actually defer fsync until it knows "quorum.size -
> > 1"
> > > has get to a certain entry offset." Why is that "quorum.size - 1"
> instead
> > > of the majority of the quorum?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Hi All,
> > > >
> > > > Just a quick update on the proposal. We have decided to move quorum
> > > > reassignment to a separate KIP:
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > > > .
> > > > The way this ties into cluster bootstrapping is complicated, so we
> felt
> > > we
> > > > needed a bit more time for validation. That leaves the core of this
> > > > proposal as quorum-based replication. If there are no further
> comments,
> > > we
> > > > will plan to start a vote later this week.
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > @Jun Rao <ju...@gmail.com>
> > > > >
> > > > > Regarding your comment about log compaction. After some deep-diving
> > > into
> > > > > this we've decided to propose a new snapshot-based log cleaning
> > > mechanism
> > > > > which would be used to replace the current compaction mechanism for
> > > this
> > > > > meta log. A new KIP will be proposed specifically for this idea.
> > > > >
> > > > > All,
> > > > >
> > > > > I've updated the KIP wiki a bit updating one config "
> > > > > election.jitter.max.ms"
> > > > > to "election.backoff.max.ms" to make it more clear about the
> usage:
> > > the
> > > > > configured value will be the upper bound of the binary exponential
> > > > backoff
> > > > > time after a failed election, before starting a new one.
> > > > >
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Thanks for the suggestions Guozhang.
> > > > > >
> > > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <
> wangguoz@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Hello Boyang,
> > > > > > >
> > > > > > > Thanks for the updated information. A few questions here:
> > > > > > >
> > > > > > > 1) Should the quorum-file also update to support multi-raft?
> > > > > > >
> > > > > > > I'm neutral about this, as we don't know yet how the multi-raft
> > > > modules
> > > > > > would behave. If
> > > > > > we have different threads operating different raft groups,
> > > > consolidating
> > > > > > the `checkpoint` files seems
> > > > > > not reasonable. We could always add `multi-quorum-file` later if
> > > > > possible.
> > > > > >
> > > > > > 2) In the previous proposal, there's fields in the
> > FetchQuorumRecords
> > > > > like
> > > > > > > latestDirtyOffset, is that dropped intentionally?
> > > > > > >
> > > > > > > I dropped the latestDirtyOffset since it is associated with the
> > log
> > > > > > compaction discussion. This is beyond this KIP scope and we could
> > > > > > potentially get a separate KIP to talk about it.
> > > > > >
> > > > > >
> > > > > > > 3) I think we also need to elaborate a bit more details
> regarding
> > > > when
> > > > > to
> > > > > > > send metadata request and discover-brokers; currently we only
> > > > discussed
> > > > > > > during bootstrap how these requests would be sent. I think the
> > > > > following
> > > > > > > scenarios would also need these requests
> > > > > > >
> > > > > > > 3.a) As long as a broker does not know the current quorum
> > > (including
> > > > > the
> > > > > > > leader and the voters), it should continue periodically ask
> other
> > > > > brokers
> > > > > > > via "metadata.
> > > > > > > 3.b) As long as a broker does not know all the current quorum
> > > voter's
> > > > > > > connections, it should continue periodically ask other brokers
> > via
> > > > > > > "discover-brokers".
> > > > > > > 3.c) When the leader's fetch timeout elapsed, it should send
> > > metadata
> > > > > > > request.
> > > > > > >
> > > > > > > Make sense, will add to the KIP.
> > > > > >
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > > > reluctanthero104@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hey all,
> > > > > > > >
> > > > > > > > follow-up on the previous email, we made some more updates:
> > > > > > > >
> > > > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to
> use
> > > > > > > multi-raft.
> > > > > > > >
> > > > > > > > 2. We add observer status into the DescribeQuorumResponse as
> we
> > > see
> > > > > it
> > > > > > > is a
> > > > > > > > low hanging fruit which is very useful for user debugging and
> > > > > > > reassignment.
> > > > > > > >
> > > > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC,
> > which
> > > > is
> > > > > > > purely
> > > > > > > > in charge of discovering broker connections in a gossip
> manner.
> > > The
> > > > > > > quorum
> > > > > > > > leader discovery is piggy-back on the Metadata RPC for the
> > topic
> > > > > > > partition
> > > > > > > > leader, which in our case is the single metadata partition
> for
> > > the
> > > > > > > version
> > > > > > > > one.
> > > > > > > >
> > > > > > > > Let me know if you have any questions.
> > > > > > > >
> > > > > > > > Boyang
> > > > > > > >
> > > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > > > reluctanthero104@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey all,
> > > > > > > > >
> > > > > > > > > Thanks for the great discussions so far. I'm posting some
> KIP
> > > > > updates
> > > > > > > > from
> > > > > > > > > our working group discussion:
> > > > > > > > >
> > > > > > > > > 1. We will be changing the core RPCs from single-raft API
> to
> > > > > > > multi-raft.
> > > > > > > > > This means all protocols will be "batch" in the first
> > version,
> > > > but
> > > > > > the
> > > > > > > > KIP
> > > > > > > > > itself only illustrates the design for a single metadata
> > topic
> > > > > > > partition.
> > > > > > > > > The reason is to "keep the door open" for future extensions
> > of
> > > > this
> > > > > > > piece
> > > > > > > > > of module such as a sharded controller or general quorum
> > based
> > > > > topic
> > > > > > > > > replication, beyond the current Kafka replication protocol.
> > > > > > > > >
> > > > > > > > > 2. We will piggy-back on the current Kafka Fetch API
> instead
> > of
> > > > > > > inventing
> > > > > > > > > a new FetchQuorumRecords RPC. The motivation is about the
> > same
> > > as
> > > > > #1
> > > > > > as
> > > > > > > > > well as making the integration work easier, instead of
> > letting
> > > > two
> > > > > > > > similar
> > > > > > > > > RPCs diverge.
> > > > > > > > >
> > > > > > > > > 3. In the EndQuorumEpoch protocol, instead of only sending
> > the
> > > > > > request
> > > > > > > to
> > > > > > > > > the most caught-up voter, we shall broadcast the
> information
> > to
> > > > all
> > > > > > > > voters,
> > > > > > > > > with a sorted voter list in descending order of their
> > > > corresponding
> > > > > > > > > replicated offset. In this way, the top voter will become a
> > > > > candidate
> > > > > > > > > immediately, while the other voters shall wait for an
> > > exponential
> > > > > > > > back-off
> > > > > > > > > to trigger elections, which helps ensure the top voter gets
> > > > > elected,
> > > > > > > and
> > > > > > > > > the election eventually happens when the top voter is not
> > > > > responsive.
> > > > > > > > >
> > > > > > > > > Please see the updated KIP and post any questions or
> concerns
> > > on
> > > > > the
> > > > > > > > > mailing thread.
> > > > > > > > >
> > > > > > > > > Boyang
> > > > > > > > >
> > > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > > >
> > > > > > > > >> Hi, Guozhang and Jason,
> > > > > > > > >>
> > > > > > > > >> Thanks for the reply. A couple of more replies.
> > > > > > > > >>
> > > > > > > > >> 102. Still not sure about this. How is the tombstone issue
> > > > > addressed
> > > > > > > in
> > > > > > > > >> the
> > > > > > > > >> non-voter and the observer.  They can die at any point and
> > > > restart
> > > > > > at
> > > > > > > an
> > > > > > > > >> arbitrary later time, and the advancing of the firstDirty
> > > offset
> > > > > and
> > > > > > > the
> > > > > > > > >> removal of the tombstone can happen independently.
> > > > > > > > >>
> > > > > > > > >> 106. I agree that it would be less confusing if we used
> > > "epoch"
> > > > > > > instead
> > > > > > > > of
> > > > > > > > >> "leader epoch" consistently.
> > > > > > > > >>
> > > > > > > > >> Jun
> > > > > > > > >>
> > > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > > > wrote:
> > > > > > > > >>
> > > > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > > > > >> >
> > > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > > > >> >
> > > > > > > > >> > > Hi, Guozhang,
> > > > > > > > >> > >
> > > > > > > > >> > > Thanks for the reply. A few more replies inlined
> below.
> > > > > > > > >> > >
> > > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> > >
> > > > > > > > >> > > > Hello Jun,
> > > > > > > > >> > > >
> > > > > > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > > > > > >> > > >
> > > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > > >> > > >
> > > > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > > > > >> > > > > 101.1 Currently, we support auto broker id
> > generation.
> > > > Is
> > > > > > this
> > > > > > > > >> > > supported
> > > > > > > > >> > > > > for bootstrap brokers?
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > > The vote ids would just be the broker ids.
> > > > > "bootstrap.servers"
> > > > > > > > >> would be
> > > > > > > > >> > > > similar to what client configs have today, where
> > > > > > "quorum.voters"
> > > > > > > > >> would
> > > > > > > > >> > be
> > > > > > > > >> > > > pre-defined config values.
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > My question was on the auto generated broker id.
> > > Currently,
> > > > > the
> > > > > > > > broker
> > > > > > > > >> > can
> > > > > > > > >> > > choose to have its broker Id auto generated. The
> > > generation
> > > > is
> > > > > > > done
> > > > > > > > >> > through
> > > > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear
> > how
> > > > the
> > > > > > > > broker
> > > > > > > > >> id
> > > > > > > > >> > is
> > > > > > > > >> > > auto generated. "quorum.voters" also can't be set
> > > statically
> > > > > if
> > > > > > > > broker
> > > > > > > > >> > ids
> > > > > > > > >> > > are auto generated.
> > > > > > > > >> > >
> > > > > > > > >> > > Jason has explained some ideas that we've discussed so
> > > far,
> > > > > the
> > > > > > > > >> reason we
> > > > > > > > >> > intentional did not include them so far is that we feel
> it
> > > is
> > > > > > > out-side
> > > > > > > > >> the
> > > > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we
> should
> > > > > > definitely
> > > > > > > > >> > address them though.
> > > > > > > > >> >
> > > > > > > > >> > On the high-level, our belief is that "joining a quorum"
> > and
> > > > > > > "joining
> > > > > > > > >> (or
> > > > > > > > >> > more specifically, registering brokers in) the cluster"
> > > would
> > > > be
> > > > > > > > >> > de-coupled a bit, where the former should be completed
> > > before
> > > > we
> > > > > > do
> > > > > > > > the
> > > > > > > > >> > latter. More specifically, assuming the quorum is
> already
> > up
> > > > and
> > > > > > > > >> running,
> > > > > > > > >> > after the newly started broker found the leader of the
> > > quorum
> > > > it
> > > > > > can
> > > > > > > > >> send a
> > > > > > > > >> > specific RegisterBroker request including its listener /
> > > > > protocol
> > > > > > /
> > > > > > > > etc,
> > > > > > > > >> > and upon handling it the leader can send back the
> uniquely
> > > > > > generated
> > > > > > > > >> broker
> > > > > > > > >> > id to the new broker, while also executing the
> > > > "startNewBroker"
> > > > > > > > >> callback as
> > > > > > > > >> > the controller.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >
> > > > > > > > >> > > > > 102. Log compaction. One weak spot of log
> compaction
> > > is
> > > > > for
> > > > > > > the
> > > > > > > > >> > > consumer
> > > > > > > > >> > > > to
> > > > > > > > >> > > > > deal with deletes. When a key is deleted, it's
> > > retained
> > > > > as a
> > > > > > > > >> > tombstone
> > > > > > > > >> > > > > first and then physically removed. If a client
> > misses
> > > > the
> > > > > > > > >> tombstone
> > > > > > > > >> > > > > (because it's physically removed), it may not be
> > able
> > > to
> > > > > > > update
> > > > > > > > >> its
> > > > > > > > >> > > > > metadata properly. The way we solve this in Kafka
> is
> > > > based
> > > > > > on
> > > > > > > a
> > > > > > > > >> > > > > configuration (log.cleaner.delete.retention.ms)
> and
> > > we
> > > > > > > expect a
> > > > > > > > >> > > consumer
> > > > > > > > >> > > > > having seen an old key to finish reading the
> > deletion
> > > > > > > tombstone
> > > > > > > > >> > within
> > > > > > > > >> > > > that
> > > > > > > > >> > > > > time. There is no strong guarantee for that since
> a
> > > > broker
> > > > > > > could
> > > > > > > > >> be
> > > > > > > > >> > > down
> > > > > > > > >> > > > > for a long time. It would be better if we can
> have a
> > > > more
> > > > > > > > reliable
> > > > > > > > >> > way
> > > > > > > > >> > > of
> > > > > > > > >> > > > > dealing with deletes.
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > > We propose to capture this in the "FirstDirtyOffset"
> > > field
> > > > > of
> > > > > > > the
> > > > > > > > >> > quorum
> > > > > > > > >> > > > record fetch response: the offset is the maximum
> > offset
> > > > that
> > > > > > log
> > > > > > > > >> > > compaction
> > > > > > > > >> > > > has reached up to. If the follower has fetched
> beyond
> > > this
> > > > > > > offset
> > > > > > > > it
> > > > > > > > >> > > means
> > > > > > > > >> > > > itself is safe hence it has seen all records up to
> > that
> > > > > > offset.
> > > > > > > On
> > > > > > > > >> > > getting
> > > > > > > > >> > > > the response, the follower can then decide if its
> end
> > > > offset
> > > > > > > > >> actually
> > > > > > > > >> > > below
> > > > > > > > >> > > > that dirty offset (and hence may miss some
> > tombstones).
> > > If
> > > > > > > that's
> > > > > > > > >> the
> > > > > > > > >> > > case:
> > > > > > > > >> > > >
> > > > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log from
> > the
> > > > very
> > > > > > > > >> beginning
> > > > > > > > >> > to
> > > > > > > > >> > > > catch up.
> > > > > > > > >> > > > 2) During that time, it would refrain itself from
> > > > answering
> > > > > > > > >> > > MetadataRequest
> > > > > > > > >> > > > from any clients.
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> > > > > addresses
> > > > > > > the
> > > > > > > > >> > issue.
> > > > > > > > >> > > Currently, the deletion tombstone is not removed
> > > immediately
> > > > > > > after a
> > > > > > > > >> > round
> > > > > > > > >> > > of cleaning. It's removed after a delay in a
> subsequent
> > > > round
> > > > > of
> > > > > > > > >> > cleaning.
> > > > > > > > >> > > Consider an example where a key insertion is at offset
> > 200
> > > > > and a
> > > > > > > > >> deletion
> > > > > > > > >> > > tombstone of the key is at 400. Initially,
> > > FirstDirtyOffset
> > > > is
> > > > > > at
> > > > > > > > >> 300. A
> > > > > > > > >> > > follower/observer fetches from offset 0  and fetches
> the
> > > key
> > > > > at
> > > > > > > > offset
> > > > > > > > >> > 200.
> > > > > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is
> at
> > > 500
> > > > > and
> > > > > > > the
> > > > > > > > >> > > tombstone at 400 is physically removed. The
> > > > follower/observer
> > > > > > > > >> continues
> > > > > > > > >> > the
> > > > > > > > >> > > fetch, but misses offset 400. It catches all the way
> to
> > > > > > > > >> FirstDirtyOffset
> > > > > > > > >> > > and declares its metadata as ready. However, its
> > metadata
> > > > > could
> > > > > > be
> > > > > > > > >> stale
> > > > > > > > >> > > since it actually misses the deletion of the key.
> > > > > > > > >> > >
> > > > > > > > >> > > Yeah good question, I should have put more details in
> my
> > > > > > > explanation
> > > > > > > > >> :)
> > > > > > > > >> >
> > > > > > > > >> > The idea is that we will adjust the log compaction for
> > this
> > > > raft
> > > > > > > based
> > > > > > > > >> > metadata log: before more details to be explained, since
> > we
> > > > have
> > > > > > two
> > > > > > > > >> types
> > > > > > > > >> > of "watermarks" here, whereas in Kafka the watermark
> > > indicates
> > > > > > where
> > > > > > > > >> every
> > > > > > > > >> > replica have replicated up to and in Raft the watermark
> > > > > indicates
> > > > > > > > where
> > > > > > > > >> the
> > > > > > > > >> > majority of replicas (here only indicating voters of the
> > > > quorum,
> > > > > > not
> > > > > > > > >> > counting observers) have replicated up to, let's call
> them
> > > > Kafka
> > > > > > > > >> watermark
> > > > > > > > >> > and Raft watermark. For this special log, we would
> > maintain
> > > > both
> > > > > > > > >> > watermarks.
> > > > > > > > >> >
> > > > > > > > >> > When log compacting on the leader, we would only compact
> > up
> > > to
> > > > > the
> > > > > > > > Kafka
> > > > > > > > >> > watermark, i.e. if there is at least one voter who have
> > not
> > > > > > > replicated
> > > > > > > > >> an
> > > > > > > > >> > entry, it would not be compacted. The "dirty-offset" is
> > the
> > > > > offset
> > > > > > > > that
> > > > > > > > >> > we've compacted up to and is communicated to other
> voters,
> > > and
> > > > > the
> > > > > > > > other
> > > > > > > > >> > voters would also compact up to this value --- i.e. the
> > > > > difference
> > > > > > > > here
> > > > > > > > >> is
> > > > > > > > >> > that instead of letting each replica doing log
> compaction
> > > > > > > > independently,
> > > > > > > > >> > we'll have the leader to decide upon which offset to
> > compact
> > > > to,
> > > > > > and
> > > > > > > > >> > propagate this value to others to follow, in a more
> > > > coordinated
> > > > > > > > manner.
> > > > > > > > >> > Also note when there are new voters joining the quorum
> who
> > > has
> > > > > not
> > > > > > > > >> > replicated up to the dirty-offset, of because of other
> > > issues
> > > > > they
> > > > > > > > >> > truncated their logs to below the dirty-offset, they'd
> > have
> > > to
> > > > > > > > >> re-bootstrap
> > > > > > > > >> > from the beginning, and during this period of time the
> > > leader
> > > > > > > learned
> > > > > > > > >> about
> > > > > > > > >> > this lagging voter would not advance the watermark (also
> > it
> > > > > would
> > > > > > > not
> > > > > > > > >> > decrement it), and hence not compacting either, until
> the
> > > > > voter(s)
> > > > > > > has
> > > > > > > > >> > caught up to that dirty-offset.
> > > > > > > > >> >
> > > > > > > > >> > So back to your example above, before the bootstrap
> voter
> > > gets
> > > > > to
> > > > > > > 300
> > > > > > > > no
> > > > > > > > >> > log compaction would happen on the leader; and until
> later
> > > > when
> > > > > > the
> > > > > > > > >> voter
> > > > > > > > >> > have got to beyond 400 and hence replicated that
> > tombstone,
> > > > the
> > > > > > log
> > > > > > > > >> > compaction would possibly get to that tombstone and
> remove
> > > it.
> > > > > Say
> > > > > > > > >> later it
> > > > > > > > >> > the leader's log compaction reaches 500, it can send
> this
> > > back
> > > > > to
> > > > > > > the
> > > > > > > > >> voter
> > > > > > > > >> > who can then also compact locally up to 500.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we
> > need
> > > > the
> > > > > > > epoch
> > > > > > > > >> > > > > corresponding to VotedId? Over time, the same
> broker
> > > Id
> > > > > > could
> > > > > > > be
> > > > > > > > >> > voted
> > > > > > > > >> > > in
> > > > > > > > >> > > > > different generations with different epoch.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > > > > "LeaderEpoch"
> > > > > > > > >> field
> > > > > > > > >> > in
> > > > > > > > >> > > > that file is corresponding to the "latest known
> leader
> > > > > epoch",
> > > > > > > not
> > > > > > > > >> the
> > > > > > > > >> > > > "current leader epoch". For example, if the current
> > > epoch
> > > > is
> > > > > > N,
> > > > > > > > and
> > > > > > > > >> > then
> > > > > > > > >> > > a
> > > > > > > > >> > > > vote-request with epoch N+1 is received and the
> voter
> > > > > granted
> > > > > > > the
> > > > > > > > >> vote
> > > > > > > > >> > > for
> > > > > > > > >> > > > it, then it means for this voter it knows the
> "latest
> > > > epoch"
> > > > > > is
> > > > > > > N
> > > > > > > > +
> > > > > > > > >> 1
> > > > > > > > >> > > > although it is unknown if that sending candidate
> will
> > > > indeed
> > > > > > > > become
> > > > > > > > >> the
> > > > > > > > >> > > new
> > > > > > > > >> > > > leader (which would only be notified via
> begin-quorum
> > > > > > request).
> > > > > > > > >> > However,
> > > > > > > > >> > > > when persisting the quorum state, we would encode
> > > > > leader-epoch
> > > > > > > to
> > > > > > > > >> N+1,
> > > > > > > > >> > > > while the leaderId to be the older leader.
> > > > > > > > >> > > >
> > > > > > > > >> > > > But now thinking about this a bit more, I feel we
> > should
> > > > use
> > > > > > two
> > > > > > > > >> > separate
> > > > > > > > >> > > > epochs, one for the "lates known" and one for the
> > > > "current"
> > > > > to
> > > > > > > > pair
> > > > > > > > >> > with
> > > > > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch
> > before
> > > > the
> > > > > > new
> > > > > > > > >> leader
> > > > > > > > >> > > is actually elected, right.
> > > > > > > > >> > >
> > > > > > > > >> > >
> > > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > > > FetchQuorumRecords
> > > > > > API
> > > > > > > to
> > > > > > > > >> > > indicate
> > > > > > > > >> > > > > that the follower has fetched from an invalid
> offset
> > > and
> > > > > > > should
> > > > > > > > >> > > truncate
> > > > > > > > >> > > > to
> > > > > > > > >> > > > > the offset/epoch indicated in the response."
> > Observers
> > > > > can't
> > > > > > > > >> truncate
> > > > > > > > >> > > > their
> > > > > > > > >> > > > > logs. What should they do with
> OFFSET_OUT_OF_RANGE?
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > I'm not sure if I understand your question?
> Observers
> > > > should
> > > > > > > still
> > > > > > > > >> be
> > > > > > > > >> > > able
> > > > > > > > >> > > > to truncate their logs as well.
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > Hmm, I thought only the quorum nodes have local logs
> and
> > > > > > observers
> > > > > > > > >> don't?
> > > > > > > > >> > >
> > > > > > > > >> > > > 107. "The leader will continue sending
> > BeginQuorumEpoch
> > > to
> > > > > > each
> > > > > > > > >> known
> > > > > > > > >> > > > voter
> > > > > > > > >> > > > > until it has received its endorsement." If a voter
> > is
> > > > down
> > > > > > > for a
> > > > > > > > >> long
> > > > > > > > >> > > > time,
> > > > > > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> > > > > overhead.
> > > > > > > > >> > Similarly,
> > > > > > > > >> > > > if a
> > > > > > > > >> > > > > follower stops sending FetchQuorumRecords, does
> the
> > > > leader
> > > > > > > keep
> > > > > > > > >> > sending
> > > > > > > > >> > > > > BeginQuorumEpoch?
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point.
> The
> > > > > > > > >> > begin-quorum-epoch
> > > > > > > > >> > > > request is for voters to quickly get the new leader
> > > > > > information;
> > > > > > > > >> > however
> > > > > > > > >> > > > even if they do not get them they can still
> eventually
> > > > learn
> > > > > > > about
> > > > > > > > >> that
> > > > > > > > >> > > > from others via gossiping FindQuorum. I think we can
> > > > adjust
> > > > > > the
> > > > > > > > >> logic
> > > > > > > > >> > to
> > > > > > > > >> > > > e.g. exponential back-off or with a limited
> > num.retries.
> > > > > > > > >> > > >
> > > > > > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > > > > > >> FetchQuorumRecords
> > > > > > > > >> > > > already, it means that follower already knows that
> the
> > > > > broker
> > > > > > is
> > > > > > > > the
> > > > > > > > >> > > > leader, and hence we can stop retrying
> > BeginQuorumEpoch;
> > > > > > however
> > > > > > > > it
> > > > > > > > >> is
> > > > > > > > >> > > > possible that after a follower sends
> > FetchQuorumRecords
> > > > > > already,
> > > > > > > > >> > suddenly
> > > > > > > > >> > > > it stops send it (possibly because it learned about
> a
> > > > higher
> > > > > > > epoch
> > > > > > > > >> > > leader),
> > > > > > > > >> > > > and hence this broker may be a "zombie" leader and
> we
> > > > > propose
> > > > > > to
> > > > > > > > use
> > > > > > > > >> > the
> > > > > > > > >> > > > fetch.timeout to let the leader to try to verify if
> it
> > > has
> > > > > > > already
> > > > > > > > >> been
> > > > > > > > >> > > > stale.
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > It just seems that we should handle these two cases
> in a
> > > > > > > consistent
> > > > > > > > >> way?
> > > > > > > > >> > >
> > > > > > > > >> > > Yes I agree, on the leader's side, the
> > FetchQuorumRecords
> > > > > from a
> > > > > > > > >> follower
> > > > > > > > >> > could mean that we no longer needs to send
> > BeginQuorumEpoch
> > > > > > anymore
> > > > > > > > ---
> > > > > > > > >> and
> > > > > > > > >> > it is already part of our current implementations in
> > > > > > > > >> >
> https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > > Thanks,
> > > > > > > > >> > >
> > > > > > > > >> > > Jun
> > > > > > > > >> > >
> > > > > > > > >> > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Jun
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > > > > wangguoz@gmail.com
> > > > > > > > >> >
> > > > > > > > >> > > > wrote:
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > > Hello Leonard,
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Thanks for your comments, I'm relying in line
> > below:
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard)
> Ge
> > <
> > > > > > > > >> > wge@confluent.io>
> > > > > > > > >> > > > > > wrote:
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > > Hi Kafka developers,
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > It's great to see this proposal and it took me
> > > some
> > > > > time
> > > > > > > to
> > > > > > > > >> > finish
> > > > > > > > >> > > > > > reading
> > > > > > > > >> > > > > > > it.
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > And I have the following questions about the
> > > > Proposal:
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > >    - How do we plan to test this design to
> > ensure
> > > > its
> > > > > > > > >> > correctness?
> > > > > > > > >> > > Or
> > > > > > > > >> > > > > > more
> > > > > > > > >> > > > > > >    broadly, how do we ensure that our new
> ‘pull’
> > > > based
> > > > > > > model
> > > > > > > > >> is
> > > > > > > > >> > > > > > functional
> > > > > > > > >> > > > > > > and
> > > > > > > > >> > > > > > >    correct given that it is different from the
> > > > > original
> > > > > > > RAFT
> > > > > > > > >> > > > > > implementation
> > > > > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > We have two planned verifications on the
> > correctness
> > > > and
> > > > > > > > >> liveness
> > > > > > > > >> > of
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > > > > >> > > > > >
> > https://github.com/guozhangwang/kafka-specification
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >    - Have we considered any sensible defaults
> for
> > > the
> > > > > > > > >> > configuration,
> > > > > > > > >> > > > i.e.
> > > > > > > > >> > > > > > >    all the election timeout, fetch time out,
> > etc.?
> > > > Or
> > > > > we
> > > > > > > > want
> > > > > > > > >> to
> > > > > > > > >> > > > leave
> > > > > > > > >> > > > > > > this to
> > > > > > > > >> > > > > > >    a later stage when we do the performance
> > > testing,
> > > > > > etc.
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > This is a good question, the reason we did not
> set
> > > any
> > > > > > > default
> > > > > > > > >> > values
> > > > > > > > >> > > > for
> > > > > > > > >> > > > > > the timeout configurations is that we think it
> may
> > > > take
> > > > > > some
> > > > > > > > >> > > > benchmarking
> > > > > > > > >> > > > > > experiments to get these defaults right. Some
> > > > high-level
> > > > > > > > >> principles
> > > > > > > > >> > > to
> > > > > > > > >> > > > > > consider: 1) the fetch.timeout should be around
> > the
> > > > same
> > > > > > > scale
> > > > > > > > >> with
> > > > > > > > >> > > zk
> > > > > > > > >> > > > > > session timeout, which is now 18 seconds by
> > default
> > > --
> > > > > in
> > > > > > > > >> practice
> > > > > > > > >> > > > we've
> > > > > > > > >> > > > > > seen unstable networks having more than 10 secs
> of
> > > > > > transient
> > > > > > > > >> > > > > connectivity,
> > > > > > > > >> > > > > > 2) the election.timeout, however, should be
> > smaller
> > > > than
> > > > > > the
> > > > > > > > >> fetch
> > > > > > > > >> > > > > timeout
> > > > > > > > >> > > > > > as is also suggested as a practical optimization
> > in
> > > > > > > > literature:
> > > > > > > > >> > > > > >
> > > > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Some more discussions can be found here:
> > > > > > > > >> > > > > >
> > > > > > > >
> > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > >    - Have we considered piggybacking
> > > > > `BeginQuorumEpoch`
> > > > > > > with
> > > > > > > > >> the
> > > > > > > > >> > `
> > > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing
> > > something
> > > > > > > obvious
> > > > > > > > >> but
> > > > > > > > >> > I
> > > > > > > > >> > > am
> > > > > > > > >> > > > > > just
> > > > > > > > >> > > > > > >    wondering why don’t we just use the
> > > `FindQuorum`
> > > > > and
> > > > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from
> leader
> > > ->
> > > > > > other
> > > > > > > > >> voter
> > > > > > > > >> > > > > > followers, while FindQuorum / Fetch are sent
> from
> > > > > follower
> > > > > > > to
> > > > > > > > >> > leader.
> > > > > > > > >> > > > > > Arguably one can eventually realize the new
> leader
> > > and
> > > > > > epoch
> > > > > > > > via
> > > > > > > > >> > > > > gossiping
> > > > > > > > >> > > > > > FindQuorum, but that could in practice require a
> > > long
> > > > > > delay.
> > > > > > > > >> > Having a
> > > > > > > > >> > > > > > leader -> other voters request helps the new
> > leader
> > > > > epoch
> > > > > > to
> > > > > > > > be
> > > > > > > > >> > > > > propagated
> > > > > > > > >> > > > > > faster under a pull model.
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > >    - And about the `FetchQuorumRecords`
> response
> > > > > schema,
> > > > > > > in
> > > > > > > > >> the
> > > > > > > > >> > > > > `Records`
> > > > > > > > >> > > > > > >    field of the response, is it just one
> record
> > or
> > > > all
> > > > > > the
> > > > > > > > >> > records
> > > > > > > > >> > > > > > starting
> > > > > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> > > > efficient
> > > > > > if
> > > > > > > we
> > > > > > > > >> sent
> > > > > > > > >> > > all
> > > > > > > > >> > > > > the
> > > > > > > > >> > > > > > >    records during the bootstrapping of the
> > > brokers.
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is just
> > the
> > > > > > > starting
> > > > > > > > >> > offset
> > > > > > > > >> > > of
> > > > > > > > >> > > > > the
> > > > > > > > >> > > > > > batch of records.
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > >    - Regarding the disruptive broker issues,
> > does
> > > > our
> > > > > > pull
> > > > > > > > >> based
> > > > > > > > >> > > > model
> > > > > > > > >> > > > > > >    suffer from it? If so, have we considered
> the
> > > > > > Pre-Vote
> > > > > > > > >> stage?
> > > > > > > > >> > If
> > > > > > > > >> > > > > not,
> > > > > > > > >> > > > > > > why?
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > The disruptive broker is stated in the original
> > Raft
> > > > > paper
> > > > > > > > >> which is
> > > > > > > > >> > > the
> > > > > > > > >> > > > > > result of the push model design. Our analysis
> > showed
> > > > > that
> > > > > > > with
> > > > > > > > >> the
> > > > > > > > >> > > pull
> > > > > > > > >> > > > > > model it is no longer an issue.
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > > Thanks a lot for putting this up, and I hope
> > that
> > > my
> > > > > > > > questions
> > > > > > > > >> > can
> > > > > > > > >> > > be
> > > > > > > > >> > > > > of
> > > > > > > > >> > > > > > > some value to make this KIP better.
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > Hope to hear from you soon!
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > Best wishes,
> > > > > > > > >> > > > > > > Leonard
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > > > > > >> cmccabe@apache.org
> > > > > > > > >> > >
> > > > > > > > >> > > > > wrote:
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > > Hi Jason,
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > I haven't had a chance to read in detail,
> but
> > I
> > > > read
> > > > > > the
> > > > > > > > >> > outline
> > > > > > > > >> > > > and
> > > > > > > > >> > > > > a
> > > > > > > > >> > > > > > > few
> > > > > > > > >> > > > > > > > things jumped out at me.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > First, for every epoch that is 32 bits
> rather
> > > than
> > > > > > 64, I
> > > > > > > > >> sort
> > > > > > > > >> > of
> > > > > > > > >> > > > > wonder
> > > > > > > > >> > > > > > > if
> > > > > > > > >> > > > > > > > that's a good long-term choice.  I keep
> > reading
> > > > > about
> > > > > > > > stuff
> > > > > > > > >> > like
> > > > > > > > >> > > > > this:
> > > > > > > > >> > > > > > > >
> > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > > > .
> > > > > > > > >> > > Obviously,
> > > > > > > > >> > > > > > that
> > > > > > > > >> > > > > > > > JIRA is about zxid, which increments much
> > faster
> > > > > than
> > > > > > we
> > > > > > > > >> expect
> > > > > > > > >> > > > these
> > > > > > > > >> > > > > > > > leader epochs to, but it would still be good
> > to
> > > > see
> > > > > > some
> > > > > > > > >> rough
> > > > > > > > >> > > > > > > calculations
> > > > > > > > >> > > > > > > > about how long 32 bits (or really, 31 bits)
> > will
> > > > > last
> > > > > > us
> > > > > > > > in
> > > > > > > > >> the
> > > > > > > > >> > > > cases
> > > > > > > > >> > > > > > > where
> > > > > > > > >> > > > > > > > we're using it, and what the space savings
> > we're
> > > > > > getting
> > > > > > > > >> really
> > > > > > > > >> > > is.
> > > > > > > > >> > > > > It
> > > > > > > > >> > > > > > > > seems like in most cases the tradeoff may
> not
> > be
> > > > > worth
> > > > > > > it?
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > Another thing I've been thinking about is
> how
> > we
> > > > do
> > > > > > > > >> > > > bootstrapping.  I
> > > > > > > > >> > > > > > > > would prefer to be in a world where
> > formatting a
> > > > new
> > > > > > > Kafka
> > > > > > > > >> node
> > > > > > > > >> > > > was a
> > > > > > > > >> > > > > > > first
> > > > > > > > >> > > > > > > > class operation explicitly initiated by the
> > > admin,
> > > > > > > rather
> > > > > > > > >> than
> > > > > > > > >> > > > > > something
> > > > > > > > >> > > > > > > > that happened implicitly when you started up
> > the
> > > > > > broker
> > > > > > > > and
> > > > > > > > >> > > things
> > > > > > > > >> > > > > > > "looked
> > > > > > > > >> > > > > > > > blank."
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > The first problem is that things can "look
> > > blank"
> > > > > > > > >> accidentally
> > > > > > > > >> > if
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > > storage system is having a bad day.  Clearly
> > in
> > > > the
> > > > > > > > non-Raft
> > > > > > > > >> > > world,
> > > > > > > > >> > > > > > this
> > > > > > > > >> > > > > > > > leads to data loss if the broker that is
> > > > (re)started
> > > > > > > this
> > > > > > > > >> way
> > > > > > > > >> > was
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > > leader for some partitions.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > The second problem is that we have a bit of
> a
> > > > > chicken
> > > > > > > and
> > > > > > > > >> egg
> > > > > > > > >> > > > problem
> > > > > > > > >> > > > > > > with
> > > > > > > > >> > > > > > > > certain configuration keys.  For example,
> > maybe
> > > > you
> > > > > > want
> > > > > > > > to
> > > > > > > > >> > > > configure
> > > > > > > > >> > > > > > > some
> > > > > > > > >> > > > > > > > connection security settings in your
> cluster,
> > > but
> > > > > you
> > > > > > > > don't
> > > > > > > > >> > want
> > > > > > > > >> > > > them
> > > > > > > > >> > > > > > to
> > > > > > > > >> > > > > > > > ever be stored in a plaintext config file.
> > (For
> > > > > > > example,
> > > > > > > > >> SCRAM
> > > > > > > > >> > > > > > > passwords,
> > > > > > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > > > > > configuration,
> > > > > > > > >> but
> > > > > > > > >> > > > that
> > > > > > > > >> > > > > > > brings
> > > > > > > > >> > > > > > > > up the chicken and egg problem.  The broker
> > > needs
> > > > to
> > > > > > be
> > > > > > > > >> > > configured
> > > > > > > > >> > > > to
> > > > > > > > >> > > > > > > know
> > > > > > > > >> > > > > > > > how to talk to you, but you need to
> configure
> > it
> > > > > > before
> > > > > > > > you
> > > > > > > > >> can
> > > > > > > > >> > > > talk
> > > > > > > > >> > > > > to
> > > > > > > > >> > > > > > > > it.  Using an external secret manager like
> > Vault
> > > > is
> > > > > > one
> > > > > > > > way
> > > > > > > > >> to
> > > > > > > > >> > > > solve
> > > > > > > > >> > > > > > > this,
> > > > > > > > >> > > > > > > > but not everyone uses an external secret
> > > manager.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > quorum.voters seems like a similar
> > configuration
> > > > > key.
> > > > > > > In
> > > > > > > > >> the
> > > > > > > > >> > > > current
> > > > > > > > >> > > > > > > KIP,
> > > > > > > > >> > > > > > > > this is only read if there is no other
> > > > configuration
> > > > > > > > >> specifying
> > > > > > > > >> > > the
> > > > > > > > >> > > > > > > quorum
> > > > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command,
> we
> > > > > > wouldn't
> > > > > > > > need
> > > > > > > > >> > this
> > > > > > > > >> > > > key
> > > > > > > > >> > > > > > > > because we could assume that there was
> always
> > > > quorum
> > > > > > > > >> > information
> > > > > > > > >> > > > > stored
> > > > > > > > >> > > > > > > > locally.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > best,
> > > > > > > > >> > > > > > > > Colin
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason
> > Gustafson
> > > > > wrote:
> > > > > > > > >> > > > > > > > > Hi All,
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > > > >> > > > > > > > .
> > > > > > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > > > > > ultimately
> > > > > > > > >> replace
> > > > > > > > >> > > > > > Zookeeper
> > > > > > > > >> > > > > > > > > as
> > > > > > > > >> > > > > > > > > documented in KIP-500. Please take a look
> > and
> > > > > share
> > > > > > > your
> > > > > > > > >> > > > thoughts.
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > A few minor notes to set the stage a
> little
> > > bit:
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > - This KIP does not specify the structure
> of
> > > the
> > > > > > > > messages
> > > > > > > > >> > used
> > > > > > > > >> > > to
> > > > > > > > >> > > > > > > > represent
> > > > > > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> > > > > internal
> > > > > > > API
> > > > > > > > >> that
> > > > > > > > >> > > will
> > > > > > > > >> > > > > be
> > > > > > > > >> > > > > > > used
> > > > > > > > >> > > > > > > > > by the controller. Expect these to come in
> > > later
> > > > > > > > >> proposals.
> > > > > > > > >> > > Here
> > > > > > > > >> > > > we
> > > > > > > > >> > > > > > are
> > > > > > > > >> > > > > > > > > primarily concerned with the replication
> > > > protocol
> > > > > > and
> > > > > > > > >> basic
> > > > > > > > >> > > > > > operational
> > > > > > > > >> > > > > > > > > mechanics.
> > > > > > > > >> > > > > > > > > - We expect many details to change as we
> get
> > > > > closer
> > > > > > to
> > > > > > > > >> > > > integration
> > > > > > > > >> > > > > > with
> > > > > > > > >> > > > > > > > > the controller. Any changes we make will
> be
> > > made
> > > > > > > either
> > > > > > > > as
> > > > > > > > >> > > > > amendments
> > > > > > > > >> > > > > > > to
> > > > > > > > >> > > > > > > > > this KIP or, in the case of larger
> changes,
> > as
> > > > new
> > > > > > > > >> proposals.
> > > > > > > > >> > > > > > > > > - We have a prototype implementation
> which I
> > > > will
> > > > > > put
> > > > > > > > >> online
> > > > > > > > >> > > > within
> > > > > > > > >> > > > > > the
> > > > > > > > >> > > > > > > > > next week which may help in understanding
> > some
> > > > > > > details.
> > > > > > > > It
> > > > > > > > >> > has
> > > > > > > > >> > > > > > > diverged a
> > > > > > > > >> > > > > > > > > little bit from our proposal, so I am
> > taking a
> > > > > > little
> > > > > > > > >> time to
> > > > > > > > >> > > > bring
> > > > > > > > >> > > > > > it
> > > > > > > > >> > > > > > > in
> > > > > > > > >> > > > > > > > > line. I'll post an update to this thread
> > when
> > > it
> > > > > is
> > > > > > > > >> available
> > > > > > > > >> > > for
> > > > > > > > >> > > > > > > review.
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > Finally, I want to mention that this
> > proposal
> > > > was
> > > > > > > > drafted
> > > > > > > > >> by
> > > > > > > > >> > > > > myself,
> > > > > > > > >> > > > > > > > Boyang
> > > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > >> > > > > > > > > Jason
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > --
> > > > > > > > >> > > > > > > Leonard Ge
> > > > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > --
> > > > > > > > >> > > > > > -- Guozhang
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > > --
> > > > > > > > >> > > > -- Guozhang
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > --
> > > > > > > > >> > -- Guozhang
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jun Rao <ju...@confluent.io>.
Hi, Jason,

Thanks for the reply.

101. Sounds good. Regarding clusterId, I am not sure storing it in the
metadata log is enough. For example, the vote request includes clusterId.
So, no one can vote until they know the clusterId. Also, it would be useful
to support the case when a voter completely loses its disk and needs to
recover.

210. There is no longer a FindQuorum request. When a follower restarts, how
does it discover the leader? Is that based on DescribeQuorum? It would be
useful to document this.

Jun

On Fri, Jul 17, 2020 at 2:15 PM Jason Gustafson <ja...@confluent.io> wrote:

> Hi Jun,
>
> Thanks for the questions.
>
> 101. I am treating some of the bootstrapping problems as out of the scope
> of this KIP. I am working on a separate proposal which addresses
> bootstrapping security credentials specifically. Here is a rough sketch of
> how I am seeing it:
>
> 1. Dynamic broker configurations including encrypted passwords will be
> persisted in the metadata log and cached in the broker's `meta.properties`
> file.
> 2. We will provide a tool which allows users to directly override the
> values in `meta.properties` without requiring access to the quorum. This
> can be used to bootstrap the credentials of the voter set itself before the
> cluster has been started.
> 3. Some dynamic config changes will only be allowed when a broker is
> online. For example, changing a truststore password dynamically would
> prevent that broker from being able to start if it were offline when the
> change was made.
> 4. I am still thinking a little bit about SCRAM credentials, but most
> likely they will be handled with an approach similar to `meta.properties`.
>
> 101.3 As for the question about `clusterId`, I think the way we would do
> this is to have the first elected leader generate a UUID and write it to
> the metadata log. Let me add some detail to the proposal about this.
>
> A few additional answers below:
>
> 203. Yes, that is correct.
>
> 204. That is a good question. What happens in this case is that all voters
> advance their epoch to the one designated by the candidate even if they
> reject its vote request. Assuming the candidate fails to be elected, the
> election will be retried until a leader emerges.
>
> 205. I had some discussion with Colin offline about this problem. I think
> the answer should be "yes," but it probably needs a little more thought.
> Handling JBOD failures is tricky. For an observer, we can replicate the
> metadata log from scratch safely in a new log dir. But if the log dir of a
> voter fails, I do not think it is generally safe to start from an empty
> state.
>
> 206. Yes, that is discussed in KIP-631 I believe.
>
> 207. Good suggestion. I will work on this.
>
>
> Thanks,
> Jason
>
>
>
>
>
> On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Jason,
> >
> > Thanks for the updated KIP. Looks good overall. A few more comments
> below.
> >
> > 101. I still don't see a section on bootstrapping related issues. It
> would
> > be useful to document if/how the following is supported.
> > 101.1 Currently, we support auto broker id generation. Is this supported
> > for bootstrap brokers?
> > 101.2 As Colin mentioned, sometimes we may need to load the security
> > credentials to be broker before it can be connected to. Could you
> provide a
> > bit more detail on how this will work?
> > 101.3 Currently, we use ZK to generate clusterId on a new cluster. With
> > Raft, how does every broker generate the same clusterId in a distributed
> > way?
> >
> > 200. It would be useful to document if the various special offsets (log
> > start offset, recovery point, HWM, etc) for the Raft log are stored in
> the
> > same existing checkpoint files or not.
> > 200.1 Since the Raft log flushes every append, does that allow us to
> > recover from a recovery point within the active segment or do we still
> need
> > to scan the full segment including the recovery point? The former can be
> > tricky since multiple records can fall into the same disk page and a
> > subsequent flush may corrupt a page with previously flushed records.
> >
> > 201. Configurations.
> > 201.1 How do the Raft brokers get security related configs for inter
> broker
> > communication? Is that based on the existing
> > inter.broker.security.protocol?
> > 201.2 We have quorum.retry.backoff.max.ms and quorum.retry.backoff.ms,
> but
> > only quorum.election.backoff.max.ms. This seems a bit inconsistent.
> >
> > 202. Metrics:
> > 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs, OutboundQueueTimeMs:
> > Are those the same as existing totalTime, requestQueueTime, localTime,
> > responseQueueTime? Could we reuse the existing ones with the tag
> > request=[request-type]?
> > 202.2. Could you explain what InboundChannelSize and OutboundChannelSize
> > are?
> > 202.3 ElectionLatencyMax/Avg: It seems that both should be windowed?
> >
> > 203. Quorum State: I assume that LeaderId will be kept consistently with
> > LeaderEpoch. For example, if a follower transitions to candidate and
> bumps
> > up LeaderEpoch, it will set leaderId to -1 and persist both in the Quorum
> > state file. Is that correct?
> >
> > 204. I was thinking about a corner case when a Raft broker is partitioned
> > off. This broker will then be in a continuous loop of bumping up the
> leader
> > epoch, but failing to get enough votes. When the partitioning is removed,
> > this broker's high leader epoch will force a leader election. I assume
> > other Raft brokers can immediately advance their leader epoch passing the
> > already bumped epoch such that leader election won't be delayed. Is that
> > right?
> >
> > 205. In a JBOD setting, could we use the existing tool to move the Raft
> log
> > from one disk to another?
> >
> > 206. The KIP doesn't mention the local metadata store derived from the
> Raft
> > log. Will that be covered in a separate KIP?
> >
> > 207. Since this is a critical component. Could we add a section on the
> > testing plan for correctness?
> >
> > 208. Performance. Do we plan to do group commit (e.g. buffer pending
> > appends during a flush and then flush all accumulated pending records
> > together in the next flush) for better throughput?
> >
> > 209. "the leader can actually defer fsync until it knows "quorum.size -
> 1"
> > has get to a certain entry offset." Why is that "quorum.size - 1" instead
> > of the majority of the quorum?
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hi All,
> > >
> > > Just a quick update on the proposal. We have decided to move quorum
> > > reassignment to a separate KIP:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > > .
> > > The way this ties into cluster bootstrapping is complicated, so we felt
> > we
> > > needed a bit more time for validation. That leaves the core of this
> > > proposal as quorum-based replication. If there are no further comments,
> > we
> > > will plan to start a vote later this week.
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > @Jun Rao <ju...@gmail.com>
> > > >
> > > > Regarding your comment about log compaction. After some deep-diving
> > into
> > > > this we've decided to propose a new snapshot-based log cleaning
> > mechanism
> > > > which would be used to replace the current compaction mechanism for
> > this
> > > > meta log. A new KIP will be proposed specifically for this idea.
> > > >
> > > > All,
> > > >
> > > > I've updated the KIP wiki a bit updating one config "
> > > > election.jitter.max.ms"
> > > > to "election.backoff.max.ms" to make it more clear about the usage:
> > the
> > > > configured value will be the upper bound of the binary exponential
> > > backoff
> > > > time after a failed election, before starting a new one.
> > > >
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> > reluctanthero104@gmail.com>
> > > > wrote:
> > > >
> > > > > Thanks for the suggestions Guozhang.
> > > > >
> > > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hello Boyang,
> > > > > >
> > > > > > Thanks for the updated information. A few questions here:
> > > > > >
> > > > > > 1) Should the quorum-file also update to support multi-raft?
> > > > > >
> > > > > > I'm neutral about this, as we don't know yet how the multi-raft
> > > modules
> > > > > would behave. If
> > > > > we have different threads operating different raft groups,
> > > consolidating
> > > > > the `checkpoint` files seems
> > > > > not reasonable. We could always add `multi-quorum-file` later if
> > > > possible.
> > > > >
> > > > > 2) In the previous proposal, there's fields in the
> FetchQuorumRecords
> > > > like
> > > > > > latestDirtyOffset, is that dropped intentionally?
> > > > > >
> > > > > > I dropped the latestDirtyOffset since it is associated with the
> log
> > > > > compaction discussion. This is beyond this KIP scope and we could
> > > > > potentially get a separate KIP to talk about it.
> > > > >
> > > > >
> > > > > > 3) I think we also need to elaborate a bit more details regarding
> > > when
> > > > to
> > > > > > send metadata request and discover-brokers; currently we only
> > > discussed
> > > > > > during bootstrap how these requests would be sent. I think the
> > > > following
> > > > > > scenarios would also need these requests
> > > > > >
> > > > > > 3.a) As long as a broker does not know the current quorum
> > (including
> > > > the
> > > > > > leader and the voters), it should continue periodically ask other
> > > > brokers
> > > > > > via "metadata.
> > > > > > 3.b) As long as a broker does not know all the current quorum
> > voter's
> > > > > > connections, it should continue periodically ask other brokers
> via
> > > > > > "discover-brokers".
> > > > > > 3.c) When the leader's fetch timeout elapsed, it should send
> > metadata
> > > > > > request.
> > > > > >
> > > > > > Make sense, will add to the KIP.
> > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > > reluctanthero104@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey all,
> > > > > > >
> > > > > > > follow-up on the previous email, we made some more updates:
> > > > > > >
> > > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > > > > > multi-raft.
> > > > > > >
> > > > > > > 2. We add observer status into the DescribeQuorumResponse as we
> > see
> > > > it
> > > > > > is a
> > > > > > > low hanging fruit which is very useful for user debugging and
> > > > > > reassignment.
> > > > > > >
> > > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC,
> which
> > > is
> > > > > > purely
> > > > > > > in charge of discovering broker connections in a gossip manner.
> > The
> > > > > > quorum
> > > > > > > leader discovery is piggy-back on the Metadata RPC for the
> topic
> > > > > > partition
> > > > > > > leader, which in our case is the single metadata partition for
> > the
> > > > > > version
> > > > > > > one.
> > > > > > >
> > > > > > > Let me know if you have any questions.
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > > reluctanthero104@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hey all,
> > > > > > > >
> > > > > > > > Thanks for the great discussions so far. I'm posting some KIP
> > > > updates
> > > > > > > from
> > > > > > > > our working group discussion:
> > > > > > > >
> > > > > > > > 1. We will be changing the core RPCs from single-raft API to
> > > > > > multi-raft.
> > > > > > > > This means all protocols will be "batch" in the first
> version,
> > > but
> > > > > the
> > > > > > > KIP
> > > > > > > > itself only illustrates the design for a single metadata
> topic
> > > > > > partition.
> > > > > > > > The reason is to "keep the door open" for future extensions
> of
> > > this
> > > > > > piece
> > > > > > > > of module such as a sharded controller or general quorum
> based
> > > > topic
> > > > > > > > replication, beyond the current Kafka replication protocol.
> > > > > > > >
> > > > > > > > 2. We will piggy-back on the current Kafka Fetch API instead
> of
> > > > > > inventing
> > > > > > > > a new FetchQuorumRecords RPC. The motivation is about the
> same
> > as
> > > > #1
> > > > > as
> > > > > > > > well as making the integration work easier, instead of
> letting
> > > two
> > > > > > > similar
> > > > > > > > RPCs diverge.
> > > > > > > >
> > > > > > > > 3. In the EndQuorumEpoch protocol, instead of only sending
> the
> > > > > request
> > > > > > to
> > > > > > > > the most caught-up voter, we shall broadcast the information
> to
> > > all
> > > > > > > voters,
> > > > > > > > with a sorted voter list in descending order of their
> > > corresponding
> > > > > > > > replicated offset. In this way, the top voter will become a
> > > > candidate
> > > > > > > > immediately, while the other voters shall wait for an
> > exponential
> > > > > > > back-off
> > > > > > > > to trigger elections, which helps ensure the top voter gets
> > > > elected,
> > > > > > and
> > > > > > > > the election eventually happens when the top voter is not
> > > > responsive.
> > > > > > > >
> > > > > > > > Please see the updated KIP and post any questions or concerns
> > on
> > > > the
> > > > > > > > mailing thread.
> > > > > > > >
> > > > > > > > Boyang
> > > > > > > >
> > > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > >
> > > > > > > >> Hi, Guozhang and Jason,
> > > > > > > >>
> > > > > > > >> Thanks for the reply. A couple of more replies.
> > > > > > > >>
> > > > > > > >> 102. Still not sure about this. How is the tombstone issue
> > > > addressed
> > > > > > in
> > > > > > > >> the
> > > > > > > >> non-voter and the observer.  They can die at any point and
> > > restart
> > > > > at
> > > > > > an
> > > > > > > >> arbitrary later time, and the advancing of the firstDirty
> > offset
> > > > and
> > > > > > the
> > > > > > > >> removal of the tombstone can happen independently.
> > > > > > > >>
> > > > > > > >> 106. I agree that it would be less confusing if we used
> > "epoch"
> > > > > > instead
> > > > > > > of
> > > > > > > >> "leader epoch" consistently.
> > > > > > > >>
> > > > > > > >> Jun
> > > > > > > >>
> > > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > > > wrote:
> > > > > > > >>
> > > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > > > >> >
> > > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > >> >
> > > > > > > >> > > Hi, Guozhang,
> > > > > > > >> > >
> > > > > > > >> > > Thanks for the reply. A few more replies inlined below.
> > > > > > > >> > >
> > > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hello Jun,
> > > > > > > >> > > >
> > > > > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > > > > >> > > >
> > > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > > > >> > > > > 101.1 Currently, we support auto broker id
> generation.
> > > Is
> > > > > this
> > > > > > > >> > > supported
> > > > > > > >> > > > > for bootstrap brokers?
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > > > The vote ids would just be the broker ids.
> > > > "bootstrap.servers"
> > > > > > > >> would be
> > > > > > > >> > > > similar to what client configs have today, where
> > > > > "quorum.voters"
> > > > > > > >> would
> > > > > > > >> > be
> > > > > > > >> > > > pre-defined config values.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > My question was on the auto generated broker id.
> > Currently,
> > > > the
> > > > > > > broker
> > > > > > > >> > can
> > > > > > > >> > > choose to have its broker Id auto generated. The
> > generation
> > > is
> > > > > > done
> > > > > > > >> > through
> > > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear
> how
> > > the
> > > > > > > broker
> > > > > > > >> id
> > > > > > > >> > is
> > > > > > > >> > > auto generated. "quorum.voters" also can't be set
> > statically
> > > > if
> > > > > > > broker
> > > > > > > >> > ids
> > > > > > > >> > > are auto generated.
> > > > > > > >> > >
> > > > > > > >> > > Jason has explained some ideas that we've discussed so
> > far,
> > > > the
> > > > > > > >> reason we
> > > > > > > >> > intentional did not include them so far is that we feel it
> > is
> > > > > > out-side
> > > > > > > >> the
> > > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> > > > > definitely
> > > > > > > >> > address them though.
> > > > > > > >> >
> > > > > > > >> > On the high-level, our belief is that "joining a quorum"
> and
> > > > > > "joining
> > > > > > > >> (or
> > > > > > > >> > more specifically, registering brokers in) the cluster"
> > would
> > > be
> > > > > > > >> > de-coupled a bit, where the former should be completed
> > before
> > > we
> > > > > do
> > > > > > > the
> > > > > > > >> > latter. More specifically, assuming the quorum is already
> up
> > > and
> > > > > > > >> running,
> > > > > > > >> > after the newly started broker found the leader of the
> > quorum
> > > it
> > > > > can
> > > > > > > >> send a
> > > > > > > >> > specific RegisterBroker request including its listener /
> > > > protocol
> > > > > /
> > > > > > > etc,
> > > > > > > >> > and upon handling it the leader can send back the uniquely
> > > > > generated
> > > > > > > >> broker
> > > > > > > >> > id to the new broker, while also executing the
> > > "startNewBroker"
> > > > > > > >> callback as
> > > > > > > >> > the controller.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >
> > > > > > > >> > > > > 102. Log compaction. One weak spot of log compaction
> > is
> > > > for
> > > > > > the
> > > > > > > >> > > consumer
> > > > > > > >> > > > to
> > > > > > > >> > > > > deal with deletes. When a key is deleted, it's
> > retained
> > > > as a
> > > > > > > >> > tombstone
> > > > > > > >> > > > > first and then physically removed. If a client
> misses
> > > the
> > > > > > > >> tombstone
> > > > > > > >> > > > > (because it's physically removed), it may not be
> able
> > to
> > > > > > update
> > > > > > > >> its
> > > > > > > >> > > > > metadata properly. The way we solve this in Kafka is
> > > based
> > > > > on
> > > > > > a
> > > > > > > >> > > > > configuration (log.cleaner.delete.retention.ms) and
> > we
> > > > > > expect a
> > > > > > > >> > > consumer
> > > > > > > >> > > > > having seen an old key to finish reading the
> deletion
> > > > > > tombstone
> > > > > > > >> > within
> > > > > > > >> > > > that
> > > > > > > >> > > > > time. There is no strong guarantee for that since a
> > > broker
> > > > > > could
> > > > > > > >> be
> > > > > > > >> > > down
> > > > > > > >> > > > > for a long time. It would be better if we can have a
> > > more
> > > > > > > reliable
> > > > > > > >> > way
> > > > > > > >> > > of
> > > > > > > >> > > > > dealing with deletes.
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > > > We propose to capture this in the "FirstDirtyOffset"
> > field
> > > > of
> > > > > > the
> > > > > > > >> > quorum
> > > > > > > >> > > > record fetch response: the offset is the maximum
> offset
> > > that
> > > > > log
> > > > > > > >> > > compaction
> > > > > > > >> > > > has reached up to. If the follower has fetched beyond
> > this
> > > > > > offset
> > > > > > > it
> > > > > > > >> > > means
> > > > > > > >> > > > itself is safe hence it has seen all records up to
> that
> > > > > offset.
> > > > > > On
> > > > > > > >> > > getting
> > > > > > > >> > > > the response, the follower can then decide if its end
> > > offset
> > > > > > > >> actually
> > > > > > > >> > > below
> > > > > > > >> > > > that dirty offset (and hence may miss some
> tombstones).
> > If
> > > > > > that's
> > > > > > > >> the
> > > > > > > >> > > case:
> > > > > > > >> > > >
> > > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log from
> the
> > > very
> > > > > > > >> beginning
> > > > > > > >> > to
> > > > > > > >> > > > catch up.
> > > > > > > >> > > > 2) During that time, it would refrain itself from
> > > answering
> > > > > > > >> > > MetadataRequest
> > > > > > > >> > > > from any clients.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> > > > addresses
> > > > > > the
> > > > > > > >> > issue.
> > > > > > > >> > > Currently, the deletion tombstone is not removed
> > immediately
> > > > > > after a
> > > > > > > >> > round
> > > > > > > >> > > of cleaning. It's removed after a delay in a subsequent
> > > round
> > > > of
> > > > > > > >> > cleaning.
> > > > > > > >> > > Consider an example where a key insertion is at offset
> 200
> > > > and a
> > > > > > > >> deletion
> > > > > > > >> > > tombstone of the key is at 400. Initially,
> > FirstDirtyOffset
> > > is
> > > > > at
> > > > > > > >> 300. A
> > > > > > > >> > > follower/observer fetches from offset 0  and fetches the
> > key
> > > > at
> > > > > > > offset
> > > > > > > >> > 200.
> > > > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at
> > 500
> > > > and
> > > > > > the
> > > > > > > >> > > tombstone at 400 is physically removed. The
> > > follower/observer
> > > > > > > >> continues
> > > > > > > >> > the
> > > > > > > >> > > fetch, but misses offset 400. It catches all the way to
> > > > > > > >> FirstDirtyOffset
> > > > > > > >> > > and declares its metadata as ready. However, its
> metadata
> > > > could
> > > > > be
> > > > > > > >> stale
> > > > > > > >> > > since it actually misses the deletion of the key.
> > > > > > > >> > >
> > > > > > > >> > > Yeah good question, I should have put more details in my
> > > > > > explanation
> > > > > > > >> :)
> > > > > > > >> >
> > > > > > > >> > The idea is that we will adjust the log compaction for
> this
> > > raft
> > > > > > based
> > > > > > > >> > metadata log: before more details to be explained, since
> we
> > > have
> > > > > two
> > > > > > > >> types
> > > > > > > >> > of "watermarks" here, whereas in Kafka the watermark
> > indicates
> > > > > where
> > > > > > > >> every
> > > > > > > >> > replica have replicated up to and in Raft the watermark
> > > > indicates
> > > > > > > where
> > > > > > > >> the
> > > > > > > >> > majority of replicas (here only indicating voters of the
> > > quorum,
> > > > > not
> > > > > > > >> > counting observers) have replicated up to, let's call them
> > > Kafka
> > > > > > > >> watermark
> > > > > > > >> > and Raft watermark. For this special log, we would
> maintain
> > > both
> > > > > > > >> > watermarks.
> > > > > > > >> >
> > > > > > > >> > When log compacting on the leader, we would only compact
> up
> > to
> > > > the
> > > > > > > Kafka
> > > > > > > >> > watermark, i.e. if there is at least one voter who have
> not
> > > > > > replicated
> > > > > > > >> an
> > > > > > > >> > entry, it would not be compacted. The "dirty-offset" is
> the
> > > > offset
> > > > > > > that
> > > > > > > >> > we've compacted up to and is communicated to other voters,
> > and
> > > > the
> > > > > > > other
> > > > > > > >> > voters would also compact up to this value --- i.e. the
> > > > difference
> > > > > > > here
> > > > > > > >> is
> > > > > > > >> > that instead of letting each replica doing log compaction
> > > > > > > independently,
> > > > > > > >> > we'll have the leader to decide upon which offset to
> compact
> > > to,
> > > > > and
> > > > > > > >> > propagate this value to others to follow, in a more
> > > coordinated
> > > > > > > manner.
> > > > > > > >> > Also note when there are new voters joining the quorum who
> > has
> > > > not
> > > > > > > >> > replicated up to the dirty-offset, of because of other
> > issues
> > > > they
> > > > > > > >> > truncated their logs to below the dirty-offset, they'd
> have
> > to
> > > > > > > >> re-bootstrap
> > > > > > > >> > from the beginning, and during this period of time the
> > leader
> > > > > > learned
> > > > > > > >> about
> > > > > > > >> > this lagging voter would not advance the watermark (also
> it
> > > > would
> > > > > > not
> > > > > > > >> > decrement it), and hence not compacting either, until the
> > > > voter(s)
> > > > > > has
> > > > > > > >> > caught up to that dirty-offset.
> > > > > > > >> >
> > > > > > > >> > So back to your example above, before the bootstrap voter
> > gets
> > > > to
> > > > > > 300
> > > > > > > no
> > > > > > > >> > log compaction would happen on the leader; and until later
> > > when
> > > > > the
> > > > > > > >> voter
> > > > > > > >> > have got to beyond 400 and hence replicated that
> tombstone,
> > > the
> > > > > log
> > > > > > > >> > compaction would possibly get to that tombstone and remove
> > it.
> > > > Say
> > > > > > > >> later it
> > > > > > > >> > the leader's log compaction reaches 500, it can send this
> > back
> > > > to
> > > > > > the
> > > > > > > >> voter
> > > > > > > >> > who can then also compact locally up to 500.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we
> need
> > > the
> > > > > > epoch
> > > > > > > >> > > > > corresponding to VotedId? Over time, the same broker
> > Id
> > > > > could
> > > > > > be
> > > > > > > >> > voted
> > > > > > > >> > > in
> > > > > > > >> > > > > different generations with different epoch.
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > > > "LeaderEpoch"
> > > > > > > >> field
> > > > > > > >> > in
> > > > > > > >> > > > that file is corresponding to the "latest known leader
> > > > epoch",
> > > > > > not
> > > > > > > >> the
> > > > > > > >> > > > "current leader epoch". For example, if the current
> > epoch
> > > is
> > > > > N,
> > > > > > > and
> > > > > > > >> > then
> > > > > > > >> > > a
> > > > > > > >> > > > vote-request with epoch N+1 is received and the voter
> > > > granted
> > > > > > the
> > > > > > > >> vote
> > > > > > > >> > > for
> > > > > > > >> > > > it, then it means for this voter it knows the "latest
> > > epoch"
> > > > > is
> > > > > > N
> > > > > > > +
> > > > > > > >> 1
> > > > > > > >> > > > although it is unknown if that sending candidate will
> > > indeed
> > > > > > > become
> > > > > > > >> the
> > > > > > > >> > > new
> > > > > > > >> > > > leader (which would only be notified via begin-quorum
> > > > > request).
> > > > > > > >> > However,
> > > > > > > >> > > > when persisting the quorum state, we would encode
> > > > leader-epoch
> > > > > > to
> > > > > > > >> N+1,
> > > > > > > >> > > > while the leaderId to be the older leader.
> > > > > > > >> > > >
> > > > > > > >> > > > But now thinking about this a bit more, I feel we
> should
> > > use
> > > > > two
> > > > > > > >> > separate
> > > > > > > >> > > > epochs, one for the "lates known" and one for the
> > > "current"
> > > > to
> > > > > > > pair
> > > > > > > >> > with
> > > > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch
> before
> > > the
> > > > > new
> > > > > > > >> leader
> > > > > > > >> > > is actually elected, right.
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > > FetchQuorumRecords
> > > > > API
> > > > > > to
> > > > > > > >> > > indicate
> > > > > > > >> > > > > that the follower has fetched from an invalid offset
> > and
> > > > > > should
> > > > > > > >> > > truncate
> > > > > > > >> > > > to
> > > > > > > >> > > > > the offset/epoch indicated in the response."
> Observers
> > > > can't
> > > > > > > >> truncate
> > > > > > > >> > > > their
> > > > > > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > I'm not sure if I understand your question? Observers
> > > should
> > > > > > still
> > > > > > > >> be
> > > > > > > >> > > able
> > > > > > > >> > > > to truncate their logs as well.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > Hmm, I thought only the quorum nodes have local logs and
> > > > > observers
> > > > > > > >> don't?
> > > > > > > >> > >
> > > > > > > >> > > > 107. "The leader will continue sending
> BeginQuorumEpoch
> > to
> > > > > each
> > > > > > > >> known
> > > > > > > >> > > > voter
> > > > > > > >> > > > > until it has received its endorsement." If a voter
> is
> > > down
> > > > > > for a
> > > > > > > >> long
> > > > > > > >> > > > time,
> > > > > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> > > > overhead.
> > > > > > > >> > Similarly,
> > > > > > > >> > > > if a
> > > > > > > >> > > > > follower stops sending FetchQuorumRecords, does the
> > > leader
> > > > > > keep
> > > > > > > >> > sending
> > > > > > > >> > > > > BeginQuorumEpoch?
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > > > > > >> > begin-quorum-epoch
> > > > > > > >> > > > request is for voters to quickly get the new leader
> > > > > information;
> > > > > > > >> > however
> > > > > > > >> > > > even if they do not get them they can still eventually
> > > learn
> > > > > > about
> > > > > > > >> that
> > > > > > > >> > > > from others via gossiping FindQuorum. I think we can
> > > adjust
> > > > > the
> > > > > > > >> logic
> > > > > > > >> > to
> > > > > > > >> > > > e.g. exponential back-off or with a limited
> num.retries.
> > > > > > > >> > > >
> > > > > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > > > > >> FetchQuorumRecords
> > > > > > > >> > > > already, it means that follower already knows that the
> > > > broker
> > > > > is
> > > > > > > the
> > > > > > > >> > > > leader, and hence we can stop retrying
> BeginQuorumEpoch;
> > > > > however
> > > > > > > it
> > > > > > > >> is
> > > > > > > >> > > > possible that after a follower sends
> FetchQuorumRecords
> > > > > already,
> > > > > > > >> > suddenly
> > > > > > > >> > > > it stops send it (possibly because it learned about a
> > > higher
> > > > > > epoch
> > > > > > > >> > > leader),
> > > > > > > >> > > > and hence this broker may be a "zombie" leader and we
> > > > propose
> > > > > to
> > > > > > > use
> > > > > > > >> > the
> > > > > > > >> > > > fetch.timeout to let the leader to try to verify if it
> > has
> > > > > > already
> > > > > > > >> been
> > > > > > > >> > > > stale.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > It just seems that we should handle these two cases in a
> > > > > > consistent
> > > > > > > >> way?
> > > > > > > >> > >
> > > > > > > >> > > Yes I agree, on the leader's side, the
> FetchQuorumRecords
> > > > from a
> > > > > > > >> follower
> > > > > > > >> > could mean that we no longer needs to send
> BeginQuorumEpoch
> > > > > anymore
> > > > > > > ---
> > > > > > > >> and
> > > > > > > >> > it is already part of our current implementations in
> > > > > > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > > Thanks,
> > > > > > > >> > >
> > > > > > > >> > > Jun
> > > > > > > >> > >
> > > > > > > >> > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > Jun
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > > > wangguoz@gmail.com
> > > > > > > >> >
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > > Hello Leonard,
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks for your comments, I'm relying in line
> below:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge
> <
> > > > > > > >> > wge@confluent.io>
> > > > > > > >> > > > > > wrote:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > > Hi Kafka developers,
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > It's great to see this proposal and it took me
> > some
> > > > time
> > > > > > to
> > > > > > > >> > finish
> > > > > > > >> > > > > > reading
> > > > > > > >> > > > > > > it.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > And I have the following questions about the
> > > Proposal:
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >    - How do we plan to test this design to
> ensure
> > > its
> > > > > > > >> > correctness?
> > > > > > > >> > > Or
> > > > > > > >> > > > > > more
> > > > > > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’
> > > based
> > > > > > model
> > > > > > > >> is
> > > > > > > >> > > > > > functional
> > > > > > > >> > > > > > > and
> > > > > > > >> > > > > > >    correct given that it is different from the
> > > > original
> > > > > > RAFT
> > > > > > > >> > > > > > implementation
> > > > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > We have two planned verifications on the
> correctness
> > > and
> > > > > > > >> liveness
> > > > > > > >> > of
> > > > > > > >> > > > the
> > > > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > > > >> > > > > >
> https://github.com/guozhangwang/kafka-specification
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >    - Have we considered any sensible defaults for
> > the
> > > > > > > >> > configuration,
> > > > > > > >> > > > i.e.
> > > > > > > >> > > > > > >    all the election timeout, fetch time out,
> etc.?
> > > Or
> > > > we
> > > > > > > want
> > > > > > > >> to
> > > > > > > >> > > > leave
> > > > > > > >> > > > > > > this to
> > > > > > > >> > > > > > >    a later stage when we do the performance
> > testing,
> > > > > etc.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > This is a good question, the reason we did not set
> > any
> > > > > > default
> > > > > > > >> > values
> > > > > > > >> > > > for
> > > > > > > >> > > > > > the timeout configurations is that we think it may
> > > take
> > > > > some
> > > > > > > >> > > > benchmarking
> > > > > > > >> > > > > > experiments to get these defaults right. Some
> > > high-level
> > > > > > > >> principles
> > > > > > > >> > > to
> > > > > > > >> > > > > > consider: 1) the fetch.timeout should be around
> the
> > > same
> > > > > > scale
> > > > > > > >> with
> > > > > > > >> > > zk
> > > > > > > >> > > > > > session timeout, which is now 18 seconds by
> default
> > --
> > > > in
> > > > > > > >> practice
> > > > > > > >> > > > we've
> > > > > > > >> > > > > > seen unstable networks having more than 10 secs of
> > > > > transient
> > > > > > > >> > > > > connectivity,
> > > > > > > >> > > > > > 2) the election.timeout, however, should be
> smaller
> > > than
> > > > > the
> > > > > > > >> fetch
> > > > > > > >> > > > > timeout
> > > > > > > >> > > > > > as is also suggested as a practical optimization
> in
> > > > > > > literature:
> > > > > > > >> > > > > >
> > > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Some more discussions can be found here:
> > > > > > > >> > > > > >
> > > > > > >
> https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > >    - Have we considered piggybacking
> > > > `BeginQuorumEpoch`
> > > > > > with
> > > > > > > >> the
> > > > > > > >> > `
> > > > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing
> > something
> > > > > > obvious
> > > > > > > >> but
> > > > > > > >> > I
> > > > > > > >> > > am
> > > > > > > >> > > > > > just
> > > > > > > >> > > > > > >    wondering why don’t we just use the
> > `FindQuorum`
> > > > and
> > > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader
> > ->
> > > > > other
> > > > > > > >> voter
> > > > > > > >> > > > > > followers, while FindQuorum / Fetch are sent from
> > > > follower
> > > > > > to
> > > > > > > >> > leader.
> > > > > > > >> > > > > > Arguably one can eventually realize the new leader
> > and
> > > > > epoch
> > > > > > > via
> > > > > > > >> > > > > gossiping
> > > > > > > >> > > > > > FindQuorum, but that could in practice require a
> > long
> > > > > delay.
> > > > > > > >> > Having a
> > > > > > > >> > > > > > leader -> other voters request helps the new
> leader
> > > > epoch
> > > > > to
> > > > > > > be
> > > > > > > >> > > > > propagated
> > > > > > > >> > > > > > faster under a pull model.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > >    - And about the `FetchQuorumRecords` response
> > > > schema,
> > > > > > in
> > > > > > > >> the
> > > > > > > >> > > > > `Records`
> > > > > > > >> > > > > > >    field of the response, is it just one record
> or
> > > all
> > > > > the
> > > > > > > >> > records
> > > > > > > >> > > > > > starting
> > > > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> > > efficient
> > > > > if
> > > > > > we
> > > > > > > >> sent
> > > > > > > >> > > all
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > >    records during the bootstrapping of the
> > brokers.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is just
> the
> > > > > > starting
> > > > > > > >> > offset
> > > > > > > >> > > of
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > batch of records.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > >    - Regarding the disruptive broker issues,
> does
> > > our
> > > > > pull
> > > > > > > >> based
> > > > > > > >> > > > model
> > > > > > > >> > > > > > >    suffer from it? If so, have we considered the
> > > > > Pre-Vote
> > > > > > > >> stage?
> > > > > > > >> > If
> > > > > > > >> > > > > not,
> > > > > > > >> > > > > > > why?
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > The disruptive broker is stated in the original
> Raft
> > > > paper
> > > > > > > >> which is
> > > > > > > >> > > the
> > > > > > > >> > > > > > result of the push model design. Our analysis
> showed
> > > > that
> > > > > > with
> > > > > > > >> the
> > > > > > > >> > > pull
> > > > > > > >> > > > > > model it is no longer an issue.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > > Thanks a lot for putting this up, and I hope
> that
> > my
> > > > > > > questions
> > > > > > > >> > can
> > > > > > > >> > > be
> > > > > > > >> > > > > of
> > > > > > > >> > > > > > > some value to make this KIP better.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Hope to hear from you soon!
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Best wishes,
> > > > > > > >> > > > > > > Leonard
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > > > > >> cmccabe@apache.org
> > > > > > > >> > >
> > > > > > > >> > > > > wrote:
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > Hi Jason,
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > I haven't had a chance to read in detail, but
> I
> > > read
> > > > > the
> > > > > > > >> > outline
> > > > > > > >> > > > and
> > > > > > > >> > > > > a
> > > > > > > >> > > > > > > few
> > > > > > > >> > > > > > > > things jumped out at me.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > First, for every epoch that is 32 bits rather
> > than
> > > > > 64, I
> > > > > > > >> sort
> > > > > > > >> > of
> > > > > > > >> > > > > wonder
> > > > > > > >> > > > > > > if
> > > > > > > >> > > > > > > > that's a good long-term choice.  I keep
> reading
> > > > about
> > > > > > > stuff
> > > > > > > >> > like
> > > > > > > >> > > > > this:
> > > > > > > >> > > > > > > >
> > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > > .
> > > > > > > >> > > Obviously,
> > > > > > > >> > > > > > that
> > > > > > > >> > > > > > > > JIRA is about zxid, which increments much
> faster
> > > > than
> > > > > we
> > > > > > > >> expect
> > > > > > > >> > > > these
> > > > > > > >> > > > > > > > leader epochs to, but it would still be good
> to
> > > see
> > > > > some
> > > > > > > >> rough
> > > > > > > >> > > > > > > calculations
> > > > > > > >> > > > > > > > about how long 32 bits (or really, 31 bits)
> will
> > > > last
> > > > > us
> > > > > > > in
> > > > > > > >> the
> > > > > > > >> > > > cases
> > > > > > > >> > > > > > > where
> > > > > > > >> > > > > > > > we're using it, and what the space savings
> we're
> > > > > getting
> > > > > > > >> really
> > > > > > > >> > > is.
> > > > > > > >> > > > > It
> > > > > > > >> > > > > > > > seems like in most cases the tradeoff may not
> be
> > > > worth
> > > > > > it?
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Another thing I've been thinking about is how
> we
> > > do
> > > > > > > >> > > > bootstrapping.  I
> > > > > > > >> > > > > > > > would prefer to be in a world where
> formatting a
> > > new
> > > > > > Kafka
> > > > > > > >> node
> > > > > > > >> > > > was a
> > > > > > > >> > > > > > > first
> > > > > > > >> > > > > > > > class operation explicitly initiated by the
> > admin,
> > > > > > rather
> > > > > > > >> than
> > > > > > > >> > > > > > something
> > > > > > > >> > > > > > > > that happened implicitly when you started up
> the
> > > > > broker
> > > > > > > and
> > > > > > > >> > > things
> > > > > > > >> > > > > > > "looked
> > > > > > > >> > > > > > > > blank."
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > The first problem is that things can "look
> > blank"
> > > > > > > >> accidentally
> > > > > > > >> > if
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > storage system is having a bad day.  Clearly
> in
> > > the
> > > > > > > non-Raft
> > > > > > > >> > > world,
> > > > > > > >> > > > > > this
> > > > > > > >> > > > > > > > leads to data loss if the broker that is
> > > (re)started
> > > > > > this
> > > > > > > >> way
> > > > > > > >> > was
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > leader for some partitions.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > The second problem is that we have a bit of a
> > > > chicken
> > > > > > and
> > > > > > > >> egg
> > > > > > > >> > > > problem
> > > > > > > >> > > > > > > with
> > > > > > > >> > > > > > > > certain configuration keys.  For example,
> maybe
> > > you
> > > > > want
> > > > > > > to
> > > > > > > >> > > > configure
> > > > > > > >> > > > > > > some
> > > > > > > >> > > > > > > > connection security settings in your cluster,
> > but
> > > > you
> > > > > > > don't
> > > > > > > >> > want
> > > > > > > >> > > > them
> > > > > > > >> > > > > > to
> > > > > > > >> > > > > > > > ever be stored in a plaintext config file.
> (For
> > > > > > example,
> > > > > > > >> SCRAM
> > > > > > > >> > > > > > > passwords,
> > > > > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > > > > configuration,
> > > > > > > >> but
> > > > > > > >> > > > that
> > > > > > > >> > > > > > > brings
> > > > > > > >> > > > > > > > up the chicken and egg problem.  The broker
> > needs
> > > to
> > > > > be
> > > > > > > >> > > configured
> > > > > > > >> > > > to
> > > > > > > >> > > > > > > know
> > > > > > > >> > > > > > > > how to talk to you, but you need to configure
> it
> > > > > before
> > > > > > > you
> > > > > > > >> can
> > > > > > > >> > > > talk
> > > > > > > >> > > > > to
> > > > > > > >> > > > > > > > it.  Using an external secret manager like
> Vault
> > > is
> > > > > one
> > > > > > > way
> > > > > > > >> to
> > > > > > > >> > > > solve
> > > > > > > >> > > > > > > this,
> > > > > > > >> > > > > > > > but not everyone uses an external secret
> > manager.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > quorum.voters seems like a similar
> configuration
> > > > key.
> > > > > > In
> > > > > > > >> the
> > > > > > > >> > > > current
> > > > > > > >> > > > > > > KIP,
> > > > > > > >> > > > > > > > this is only read if there is no other
> > > configuration
> > > > > > > >> specifying
> > > > > > > >> > > the
> > > > > > > >> > > > > > > quorum
> > > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> > > > > wouldn't
> > > > > > > need
> > > > > > > >> > this
> > > > > > > >> > > > key
> > > > > > > >> > > > > > > > because we could assume that there was always
> > > quorum
> > > > > > > >> > information
> > > > > > > >> > > > > stored
> > > > > > > >> > > > > > > > locally.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > best,
> > > > > > > >> > > > > > > > Colin
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason
> Gustafson
> > > > wrote:
> > > > > > > >> > > > > > > > > Hi All,
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > > >> > > > > > > > .
> > > > > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > > > > ultimately
> > > > > > > >> replace
> > > > > > > >> > > > > > Zookeeper
> > > > > > > >> > > > > > > > > as
> > > > > > > >> > > > > > > > > documented in KIP-500. Please take a look
> and
> > > > share
> > > > > > your
> > > > > > > >> > > > thoughts.
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > A few minor notes to set the stage a little
> > bit:
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > - This KIP does not specify the structure of
> > the
> > > > > > > messages
> > > > > > > >> > used
> > > > > > > >> > > to
> > > > > > > >> > > > > > > > represent
> > > > > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> > > > internal
> > > > > > API
> > > > > > > >> that
> > > > > > > >> > > will
> > > > > > > >> > > > > be
> > > > > > > >> > > > > > > used
> > > > > > > >> > > > > > > > > by the controller. Expect these to come in
> > later
> > > > > > > >> proposals.
> > > > > > > >> > > Here
> > > > > > > >> > > > we
> > > > > > > >> > > > > > are
> > > > > > > >> > > > > > > > > primarily concerned with the replication
> > > protocol
> > > > > and
> > > > > > > >> basic
> > > > > > > >> > > > > > operational
> > > > > > > >> > > > > > > > > mechanics.
> > > > > > > >> > > > > > > > > - We expect many details to change as we get
> > > > closer
> > > > > to
> > > > > > > >> > > > integration
> > > > > > > >> > > > > > with
> > > > > > > >> > > > > > > > > the controller. Any changes we make will be
> > made
> > > > > > either
> > > > > > > as
> > > > > > > >> > > > > amendments
> > > > > > > >> > > > > > > to
> > > > > > > >> > > > > > > > > this KIP or, in the case of larger changes,
> as
> > > new
> > > > > > > >> proposals.
> > > > > > > >> > > > > > > > > - We have a prototype implementation which I
> > > will
> > > > > put
> > > > > > > >> online
> > > > > > > >> > > > within
> > > > > > > >> > > > > > the
> > > > > > > >> > > > > > > > > next week which may help in understanding
> some
> > > > > > details.
> > > > > > > It
> > > > > > > >> > has
> > > > > > > >> > > > > > > diverged a
> > > > > > > >> > > > > > > > > little bit from our proposal, so I am
> taking a
> > > > > little
> > > > > > > >> time to
> > > > > > > >> > > > bring
> > > > > > > >> > > > > > it
> > > > > > > >> > > > > > > in
> > > > > > > >> > > > > > > > > line. I'll post an update to this thread
> when
> > it
> > > > is
> > > > > > > >> available
> > > > > > > >> > > for
> > > > > > > >> > > > > > > review.
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > Finally, I want to mention that this
> proposal
> > > was
> > > > > > > drafted
> > > > > > > >> by
> > > > > > > >> > > > > myself,
> > > > > > > >> > > > > > > > Boyang
> > > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > Thanks,
> > > > > > > >> > > > > > > > > Jason
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > --
> > > > > > > >> > > > > > > Leonard Ge
> > > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > --
> > > > > > > >> > > > > > -- Guozhang
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > > --
> > > > > > > >> > > > -- Guozhang
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > --
> > > > > > > >> > -- Guozhang
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Jun,

Thanks for the questions.

101. I am treating some of the bootstrapping problems as out of the scope
of this KIP. I am working on a separate proposal which addresses
bootstrapping security credentials specifically. Here is a rough sketch of
how I am seeing it:

1. Dynamic broker configurations including encrypted passwords will be
persisted in the metadata log and cached in the broker's `meta.properties`
file.
2. We will provide a tool which allows users to directly override the
values in `meta.properties` without requiring access to the quorum. This
can be used to bootstrap the credentials of the voter set itself before the
cluster has been started.
3. Some dynamic config changes will only be allowed when a broker is
online. For example, changing a truststore password dynamically would
prevent that broker from being able to start if it were offline when the
change was made.
4. I am still thinking a little bit about SCRAM credentials, but most
likely they will be handled with an approach similar to `meta.properties`.

101.3 As for the question about `clusterId`, I think the way we would do
this is to have the first elected leader generate a UUID and write it to
the metadata log. Let me add some detail to the proposal about this.

A few additional answers below:

203. Yes, that is correct.

204. That is a good question. What happens in this case is that all voters
advance their epoch to the one designated by the candidate even if they
reject its vote request. Assuming the candidate fails to be elected, the
election will be retried until a leader emerges.

205. I had some discussion with Colin offline about this problem. I think
the answer should be "yes," but it probably needs a little more thought.
Handling JBOD failures is tricky. For an observer, we can replicate the
metadata log from scratch safely in a new log dir. But if the log dir of a
voter fails, I do not think it is generally safe to start from an empty
state.

206. Yes, that is discussed in KIP-631 I believe.

207. Good suggestion. I will work on this.


Thanks,
Jason





On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Jason,
>
> Thanks for the updated KIP. Looks good overall. A few more comments below.
>
> 101. I still don't see a section on bootstrapping related issues. It would
> be useful to document if/how the following is supported.
> 101.1 Currently, we support auto broker id generation. Is this supported
> for bootstrap brokers?
> 101.2 As Colin mentioned, sometimes we may need to load the security
> credentials to be broker before it can be connected to. Could you provide a
> bit more detail on how this will work?
> 101.3 Currently, we use ZK to generate clusterId on a new cluster. With
> Raft, how does every broker generate the same clusterId in a distributed
> way?
>
> 200. It would be useful to document if the various special offsets (log
> start offset, recovery point, HWM, etc) for the Raft log are stored in the
> same existing checkpoint files or not.
> 200.1 Since the Raft log flushes every append, does that allow us to
> recover from a recovery point within the active segment or do we still need
> to scan the full segment including the recovery point? The former can be
> tricky since multiple records can fall into the same disk page and a
> subsequent flush may corrupt a page with previously flushed records.
>
> 201. Configurations.
> 201.1 How do the Raft brokers get security related configs for inter broker
> communication? Is that based on the existing
> inter.broker.security.protocol?
> 201.2 We have quorum.retry.backoff.max.ms and quorum.retry.backoff.ms, but
> only quorum.election.backoff.max.ms. This seems a bit inconsistent.
>
> 202. Metrics:
> 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs, OutboundQueueTimeMs:
> Are those the same as existing totalTime, requestQueueTime, localTime,
> responseQueueTime? Could we reuse the existing ones with the tag
> request=[request-type]?
> 202.2. Could you explain what InboundChannelSize and OutboundChannelSize
> are?
> 202.3 ElectionLatencyMax/Avg: It seems that both should be windowed?
>
> 203. Quorum State: I assume that LeaderId will be kept consistently with
> LeaderEpoch. For example, if a follower transitions to candidate and bumps
> up LeaderEpoch, it will set leaderId to -1 and persist both in the Quorum
> state file. Is that correct?
>
> 204. I was thinking about a corner case when a Raft broker is partitioned
> off. This broker will then be in a continuous loop of bumping up the leader
> epoch, but failing to get enough votes. When the partitioning is removed,
> this broker's high leader epoch will force a leader election. I assume
> other Raft brokers can immediately advance their leader epoch passing the
> already bumped epoch such that leader election won't be delayed. Is that
> right?
>
> 205. In a JBOD setting, could we use the existing tool to move the Raft log
> from one disk to another?
>
> 206. The KIP doesn't mention the local metadata store derived from the Raft
> log. Will that be covered in a separate KIP?
>
> 207. Since this is a critical component. Could we add a section on the
> testing plan for correctness?
>
> 208. Performance. Do we plan to do group commit (e.g. buffer pending
> appends during a flush and then flush all accumulated pending records
> together in the next flush) for better throughput?
>
> 209. "the leader can actually defer fsync until it knows "quorum.size - 1"
> has get to a certain entry offset." Why is that "quorum.size - 1" instead
> of the majority of the quorum?
>
> Thanks,
>
> Jun
>
> On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi All,
> >
> > Just a quick update on the proposal. We have decided to move quorum
> > reassignment to a separate KIP:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > .
> > The way this ties into cluster bootstrapping is complicated, so we felt
> we
> > needed a bit more time for validation. That leaves the core of this
> > proposal as quorum-based replication. If there are no further comments,
> we
> > will plan to start a vote later this week.
> >
> > Thanks,
> > Jason
> >
> > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > @Jun Rao <ju...@gmail.com>
> > >
> > > Regarding your comment about log compaction. After some deep-diving
> into
> > > this we've decided to propose a new snapshot-based log cleaning
> mechanism
> > > which would be used to replace the current compaction mechanism for
> this
> > > meta log. A new KIP will be proposed specifically for this idea.
> > >
> > > All,
> > >
> > > I've updated the KIP wiki a bit updating one config "
> > > election.jitter.max.ms"
> > > to "election.backoff.max.ms" to make it more clear about the usage:
> the
> > > configured value will be the upper bound of the binary exponential
> > backoff
> > > time after a failed election, before starting a new one.
> > >
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> reluctanthero104@gmail.com>
> > > wrote:
> > >
> > > > Thanks for the suggestions Guozhang.
> > > >
> > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hello Boyang,
> > > > >
> > > > > Thanks for the updated information. A few questions here:
> > > > >
> > > > > 1) Should the quorum-file also update to support multi-raft?
> > > > >
> > > > > I'm neutral about this, as we don't know yet how the multi-raft
> > modules
> > > > would behave. If
> > > > we have different threads operating different raft groups,
> > consolidating
> > > > the `checkpoint` files seems
> > > > not reasonable. We could always add `multi-quorum-file` later if
> > > possible.
> > > >
> > > > 2) In the previous proposal, there's fields in the FetchQuorumRecords
> > > like
> > > > > latestDirtyOffset, is that dropped intentionally?
> > > > >
> > > > > I dropped the latestDirtyOffset since it is associated with the log
> > > > compaction discussion. This is beyond this KIP scope and we could
> > > > potentially get a separate KIP to talk about it.
> > > >
> > > >
> > > > > 3) I think we also need to elaborate a bit more details regarding
> > when
> > > to
> > > > > send metadata request and discover-brokers; currently we only
> > discussed
> > > > > during bootstrap how these requests would be sent. I think the
> > > following
> > > > > scenarios would also need these requests
> > > > >
> > > > > 3.a) As long as a broker does not know the current quorum
> (including
> > > the
> > > > > leader and the voters), it should continue periodically ask other
> > > brokers
> > > > > via "metadata.
> > > > > 3.b) As long as a broker does not know all the current quorum
> voter's
> > > > > connections, it should continue periodically ask other brokers via
> > > > > "discover-brokers".
> > > > > 3.c) When the leader's fetch timeout elapsed, it should send
> metadata
> > > > > request.
> > > > >
> > > > > Make sense, will add to the KIP.
> > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey all,
> > > > > >
> > > > > > follow-up on the previous email, we made some more updates:
> > > > > >
> > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > > > > multi-raft.
> > > > > >
> > > > > > 2. We add observer status into the DescribeQuorumResponse as we
> see
> > > it
> > > > > is a
> > > > > > low hanging fruit which is very useful for user debugging and
> > > > > reassignment.
> > > > > >
> > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which
> > is
> > > > > purely
> > > > > > in charge of discovering broker connections in a gossip manner.
> The
> > > > > quorum
> > > > > > leader discovery is piggy-back on the Metadata RPC for the topic
> > > > > partition
> > > > > > leader, which in our case is the single metadata partition for
> the
> > > > > version
> > > > > > one.
> > > > > >
> > > > > > Let me know if you have any questions.
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > reluctanthero104@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey all,
> > > > > > >
> > > > > > > Thanks for the great discussions so far. I'm posting some KIP
> > > updates
> > > > > > from
> > > > > > > our working group discussion:
> > > > > > >
> > > > > > > 1. We will be changing the core RPCs from single-raft API to
> > > > > multi-raft.
> > > > > > > This means all protocols will be "batch" in the first version,
> > but
> > > > the
> > > > > > KIP
> > > > > > > itself only illustrates the design for a single metadata topic
> > > > > partition.
> > > > > > > The reason is to "keep the door open" for future extensions of
> > this
> > > > > piece
> > > > > > > of module such as a sharded controller or general quorum based
> > > topic
> > > > > > > replication, beyond the current Kafka replication protocol.
> > > > > > >
> > > > > > > 2. We will piggy-back on the current Kafka Fetch API instead of
> > > > > inventing
> > > > > > > a new FetchQuorumRecords RPC. The motivation is about the same
> as
> > > #1
> > > > as
> > > > > > > well as making the integration work easier, instead of letting
> > two
> > > > > > similar
> > > > > > > RPCs diverge.
> > > > > > >
> > > > > > > 3. In the EndQuorumEpoch protocol, instead of only sending the
> > > > request
> > > > > to
> > > > > > > the most caught-up voter, we shall broadcast the information to
> > all
> > > > > > voters,
> > > > > > > with a sorted voter list in descending order of their
> > corresponding
> > > > > > > replicated offset. In this way, the top voter will become a
> > > candidate
> > > > > > > immediately, while the other voters shall wait for an
> exponential
> > > > > > back-off
> > > > > > > to trigger elections, which helps ensure the top voter gets
> > > elected,
> > > > > and
> > > > > > > the election eventually happens when the top voter is not
> > > responsive.
> > > > > > >
> > > > > > > Please see the updated KIP and post any questions or concerns
> on
> > > the
> > > > > > > mailing thread.
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > >
> > > > > > >> Hi, Guozhang and Jason,
> > > > > > >>
> > > > > > >> Thanks for the reply. A couple of more replies.
> > > > > > >>
> > > > > > >> 102. Still not sure about this. How is the tombstone issue
> > > addressed
> > > > > in
> > > > > > >> the
> > > > > > >> non-voter and the observer.  They can die at any point and
> > restart
> > > > at
> > > > > an
> > > > > > >> arbitrary later time, and the advancing of the firstDirty
> offset
> > > and
> > > > > the
> > > > > > >> removal of the tombstone can happen independently.
> > > > > > >>
> > > > > > >> 106. I agree that it would be less confusing if we used
> "epoch"
> > > > > instead
> > > > > > of
> > > > > > >> "leader epoch" consistently.
> > > > > > >>
> > > > > > >> Jun
> > > > > > >>
> > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > > >> >
> > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > >> >
> > > > > > >> > > Hi, Guozhang,
> > > > > > >> > >
> > > > > > >> > > Thanks for the reply. A few more replies inlined below.
> > > > > > >> > >
> > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Hello Jun,
> > > > > > >> > > >
> > > > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > > > >> > > >
> > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > > >> > > > > 101.1 Currently, we support auto broker id generation.
> > Is
> > > > this
> > > > > > >> > > supported
> > > > > > >> > > > > for bootstrap brokers?
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > The vote ids would just be the broker ids.
> > > "bootstrap.servers"
> > > > > > >> would be
> > > > > > >> > > > similar to what client configs have today, where
> > > > "quorum.voters"
> > > > > > >> would
> > > > > > >> > be
> > > > > > >> > > > pre-defined config values.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > My question was on the auto generated broker id.
> Currently,
> > > the
> > > > > > broker
> > > > > > >> > can
> > > > > > >> > > choose to have its broker Id auto generated. The
> generation
> > is
> > > > > done
> > > > > > >> > through
> > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how
> > the
> > > > > > broker
> > > > > > >> id
> > > > > > >> > is
> > > > > > >> > > auto generated. "quorum.voters" also can't be set
> statically
> > > if
> > > > > > broker
> > > > > > >> > ids
> > > > > > >> > > are auto generated.
> > > > > > >> > >
> > > > > > >> > > Jason has explained some ideas that we've discussed so
> far,
> > > the
> > > > > > >> reason we
> > > > > > >> > intentional did not include them so far is that we feel it
> is
> > > > > out-side
> > > > > > >> the
> > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> > > > definitely
> > > > > > >> > address them though.
> > > > > > >> >
> > > > > > >> > On the high-level, our belief is that "joining a quorum" and
> > > > > "joining
> > > > > > >> (or
> > > > > > >> > more specifically, registering brokers in) the cluster"
> would
> > be
> > > > > > >> > de-coupled a bit, where the former should be completed
> before
> > we
> > > > do
> > > > > > the
> > > > > > >> > latter. More specifically, assuming the quorum is already up
> > and
> > > > > > >> running,
> > > > > > >> > after the newly started broker found the leader of the
> quorum
> > it
> > > > can
> > > > > > >> send a
> > > > > > >> > specific RegisterBroker request including its listener /
> > > protocol
> > > > /
> > > > > > etc,
> > > > > > >> > and upon handling it the leader can send back the uniquely
> > > > generated
> > > > > > >> broker
> > > > > > >> > id to the new broker, while also executing the
> > "startNewBroker"
> > > > > > >> callback as
> > > > > > >> > the controller.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >
> > > > > > >> > > > > 102. Log compaction. One weak spot of log compaction
> is
> > > for
> > > > > the
> > > > > > >> > > consumer
> > > > > > >> > > > to
> > > > > > >> > > > > deal with deletes. When a key is deleted, it's
> retained
> > > as a
> > > > > > >> > tombstone
> > > > > > >> > > > > first and then physically removed. If a client misses
> > the
> > > > > > >> tombstone
> > > > > > >> > > > > (because it's physically removed), it may not be able
> to
> > > > > update
> > > > > > >> its
> > > > > > >> > > > > metadata properly. The way we solve this in Kafka is
> > based
> > > > on
> > > > > a
> > > > > > >> > > > > configuration (log.cleaner.delete.retention.ms) and
> we
> > > > > expect a
> > > > > > >> > > consumer
> > > > > > >> > > > > having seen an old key to finish reading the deletion
> > > > > tombstone
> > > > > > >> > within
> > > > > > >> > > > that
> > > > > > >> > > > > time. There is no strong guarantee for that since a
> > broker
> > > > > could
> > > > > > >> be
> > > > > > >> > > down
> > > > > > >> > > > > for a long time. It would be better if we can have a
> > more
> > > > > > reliable
> > > > > > >> > way
> > > > > > >> > > of
> > > > > > >> > > > > dealing with deletes.
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > We propose to capture this in the "FirstDirtyOffset"
> field
> > > of
> > > > > the
> > > > > > >> > quorum
> > > > > > >> > > > record fetch response: the offset is the maximum offset
> > that
> > > > log
> > > > > > >> > > compaction
> > > > > > >> > > > has reached up to. If the follower has fetched beyond
> this
> > > > > offset
> > > > > > it
> > > > > > >> > > means
> > > > > > >> > > > itself is safe hence it has seen all records up to that
> > > > offset.
> > > > > On
> > > > > > >> > > getting
> > > > > > >> > > > the response, the follower can then decide if its end
> > offset
> > > > > > >> actually
> > > > > > >> > > below
> > > > > > >> > > > that dirty offset (and hence may miss some tombstones).
> If
> > > > > that's
> > > > > > >> the
> > > > > > >> > > case:
> > > > > > >> > > >
> > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log from the
> > very
> > > > > > >> beginning
> > > > > > >> > to
> > > > > > >> > > > catch up.
> > > > > > >> > > > 2) During that time, it would refrain itself from
> > answering
> > > > > > >> > > MetadataRequest
> > > > > > >> > > > from any clients.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> > > addresses
> > > > > the
> > > > > > >> > issue.
> > > > > > >> > > Currently, the deletion tombstone is not removed
> immediately
> > > > > after a
> > > > > > >> > round
> > > > > > >> > > of cleaning. It's removed after a delay in a subsequent
> > round
> > > of
> > > > > > >> > cleaning.
> > > > > > >> > > Consider an example where a key insertion is at offset 200
> > > and a
> > > > > > >> deletion
> > > > > > >> > > tombstone of the key is at 400. Initially,
> FirstDirtyOffset
> > is
> > > > at
> > > > > > >> 300. A
> > > > > > >> > > follower/observer fetches from offset 0  and fetches the
> key
> > > at
> > > > > > offset
> > > > > > >> > 200.
> > > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at
> 500
> > > and
> > > > > the
> > > > > > >> > > tombstone at 400 is physically removed. The
> > follower/observer
> > > > > > >> continues
> > > > > > >> > the
> > > > > > >> > > fetch, but misses offset 400. It catches all the way to
> > > > > > >> FirstDirtyOffset
> > > > > > >> > > and declares its metadata as ready. However, its metadata
> > > could
> > > > be
> > > > > > >> stale
> > > > > > >> > > since it actually misses the deletion of the key.
> > > > > > >> > >
> > > > > > >> > > Yeah good question, I should have put more details in my
> > > > > explanation
> > > > > > >> :)
> > > > > > >> >
> > > > > > >> > The idea is that we will adjust the log compaction for this
> > raft
> > > > > based
> > > > > > >> > metadata log: before more details to be explained, since we
> > have
> > > > two
> > > > > > >> types
> > > > > > >> > of "watermarks" here, whereas in Kafka the watermark
> indicates
> > > > where
> > > > > > >> every
> > > > > > >> > replica have replicated up to and in Raft the watermark
> > > indicates
> > > > > > where
> > > > > > >> the
> > > > > > >> > majority of replicas (here only indicating voters of the
> > quorum,
> > > > not
> > > > > > >> > counting observers) have replicated up to, let's call them
> > Kafka
> > > > > > >> watermark
> > > > > > >> > and Raft watermark. For this special log, we would maintain
> > both
> > > > > > >> > watermarks.
> > > > > > >> >
> > > > > > >> > When log compacting on the leader, we would only compact up
> to
> > > the
> > > > > > Kafka
> > > > > > >> > watermark, i.e. if there is at least one voter who have not
> > > > > replicated
> > > > > > >> an
> > > > > > >> > entry, it would not be compacted. The "dirty-offset" is the
> > > offset
> > > > > > that
> > > > > > >> > we've compacted up to and is communicated to other voters,
> and
> > > the
> > > > > > other
> > > > > > >> > voters would also compact up to this value --- i.e. the
> > > difference
> > > > > > here
> > > > > > >> is
> > > > > > >> > that instead of letting each replica doing log compaction
> > > > > > independently,
> > > > > > >> > we'll have the leader to decide upon which offset to compact
> > to,
> > > > and
> > > > > > >> > propagate this value to others to follow, in a more
> > coordinated
> > > > > > manner.
> > > > > > >> > Also note when there are new voters joining the quorum who
> has
> > > not
> > > > > > >> > replicated up to the dirty-offset, of because of other
> issues
> > > they
> > > > > > >> > truncated their logs to below the dirty-offset, they'd have
> to
> > > > > > >> re-bootstrap
> > > > > > >> > from the beginning, and during this period of time the
> leader
> > > > > learned
> > > > > > >> about
> > > > > > >> > this lagging voter would not advance the watermark (also it
> > > would
> > > > > not
> > > > > > >> > decrement it), and hence not compacting either, until the
> > > voter(s)
> > > > > has
> > > > > > >> > caught up to that dirty-offset.
> > > > > > >> >
> > > > > > >> > So back to your example above, before the bootstrap voter
> gets
> > > to
> > > > > 300
> > > > > > no
> > > > > > >> > log compaction would happen on the leader; and until later
> > when
> > > > the
> > > > > > >> voter
> > > > > > >> > have got to beyond 400 and hence replicated that tombstone,
> > the
> > > > log
> > > > > > >> > compaction would possibly get to that tombstone and remove
> it.
> > > Say
> > > > > > >> later it
> > > > > > >> > the leader's log compaction reaches 500, it can send this
> back
> > > to
> > > > > the
> > > > > > >> voter
> > > > > > >> > who can then also compact locally up to 500.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we need
> > the
> > > > > epoch
> > > > > > >> > > > > corresponding to VotedId? Over time, the same broker
> Id
> > > > could
> > > > > be
> > > > > > >> > voted
> > > > > > >> > > in
> > > > > > >> > > > > different generations with different epoch.
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > > "LeaderEpoch"
> > > > > > >> field
> > > > > > >> > in
> > > > > > >> > > > that file is corresponding to the "latest known leader
> > > epoch",
> > > > > not
> > > > > > >> the
> > > > > > >> > > > "current leader epoch". For example, if the current
> epoch
> > is
> > > > N,
> > > > > > and
> > > > > > >> > then
> > > > > > >> > > a
> > > > > > >> > > > vote-request with epoch N+1 is received and the voter
> > > granted
> > > > > the
> > > > > > >> vote
> > > > > > >> > > for
> > > > > > >> > > > it, then it means for this voter it knows the "latest
> > epoch"
> > > > is
> > > > > N
> > > > > > +
> > > > > > >> 1
> > > > > > >> > > > although it is unknown if that sending candidate will
> > indeed
> > > > > > become
> > > > > > >> the
> > > > > > >> > > new
> > > > > > >> > > > leader (which would only be notified via begin-quorum
> > > > request).
> > > > > > >> > However,
> > > > > > >> > > > when persisting the quorum state, we would encode
> > > leader-epoch
> > > > > to
> > > > > > >> N+1,
> > > > > > >> > > > while the leaderId to be the older leader.
> > > > > > >> > > >
> > > > > > >> > > > But now thinking about this a bit more, I feel we should
> > use
> > > > two
> > > > > > >> > separate
> > > > > > >> > > > epochs, one for the "lates known" and one for the
> > "current"
> > > to
> > > > > > pair
> > > > > > >> > with
> > > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch before
> > the
> > > > new
> > > > > > >> leader
> > > > > > >> > > is actually elected, right.
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > FetchQuorumRecords
> > > > API
> > > > > to
> > > > > > >> > > indicate
> > > > > > >> > > > > that the follower has fetched from an invalid offset
> and
> > > > > should
> > > > > > >> > > truncate
> > > > > > >> > > > to
> > > > > > >> > > > > the offset/epoch indicated in the response." Observers
> > > can't
> > > > > > >> truncate
> > > > > > >> > > > their
> > > > > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > I'm not sure if I understand your question? Observers
> > should
> > > > > still
> > > > > > >> be
> > > > > > >> > > able
> > > > > > >> > > > to truncate their logs as well.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > Hmm, I thought only the quorum nodes have local logs and
> > > > observers
> > > > > > >> don't?
> > > > > > >> > >
> > > > > > >> > > > 107. "The leader will continue sending BeginQuorumEpoch
> to
> > > > each
> > > > > > >> known
> > > > > > >> > > > voter
> > > > > > >> > > > > until it has received its endorsement." If a voter is
> > down
> > > > > for a
> > > > > > >> long
> > > > > > >> > > > time,
> > > > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> > > overhead.
> > > > > > >> > Similarly,
> > > > > > >> > > > if a
> > > > > > >> > > > > follower stops sending FetchQuorumRecords, does the
> > leader
> > > > > keep
> > > > > > >> > sending
> > > > > > >> > > > > BeginQuorumEpoch?
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > > > > >> > begin-quorum-epoch
> > > > > > >> > > > request is for voters to quickly get the new leader
> > > > information;
> > > > > > >> > however
> > > > > > >> > > > even if they do not get them they can still eventually
> > learn
> > > > > about
> > > > > > >> that
> > > > > > >> > > > from others via gossiping FindQuorum. I think we can
> > adjust
> > > > the
> > > > > > >> logic
> > > > > > >> > to
> > > > > > >> > > > e.g. exponential back-off or with a limited num.retries.
> > > > > > >> > > >
> > > > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > > > >> FetchQuorumRecords
> > > > > > >> > > > already, it means that follower already knows that the
> > > broker
> > > > is
> > > > > > the
> > > > > > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch;
> > > > however
> > > > > > it
> > > > > > >> is
> > > > > > >> > > > possible that after a follower sends FetchQuorumRecords
> > > > already,
> > > > > > >> > suddenly
> > > > > > >> > > > it stops send it (possibly because it learned about a
> > higher
> > > > > epoch
> > > > > > >> > > leader),
> > > > > > >> > > > and hence this broker may be a "zombie" leader and we
> > > propose
> > > > to
> > > > > > use
> > > > > > >> > the
> > > > > > >> > > > fetch.timeout to let the leader to try to verify if it
> has
> > > > > already
> > > > > > >> been
> > > > > > >> > > > stale.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > It just seems that we should handle these two cases in a
> > > > > consistent
> > > > > > >> way?
> > > > > > >> > >
> > > > > > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords
> > > from a
> > > > > > >> follower
> > > > > > >> > could mean that we no longer needs to send BeginQuorumEpoch
> > > > anymore
> > > > > > ---
> > > > > > >> and
> > > > > > >> > it is already part of our current implementations in
> > > > > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > Thanks,
> > > > > > >> > >
> > > > > > >> > > Jun
> > > > > > >> > >
> > > > > > >> > > >
> > > > > > >> > > > >
> > > > > > >> > > > > Jun
> > > > > > >> > > > >
> > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > >> >
> > > > > > >> > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hello Leonard,
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks for your comments, I'm relying in line below:
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > > > > > >> > wge@confluent.io>
> > > > > > >> > > > > > wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Hi Kafka developers,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > It's great to see this proposal and it took me
> some
> > > time
> > > > > to
> > > > > > >> > finish
> > > > > > >> > > > > > reading
> > > > > > >> > > > > > > it.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > And I have the following questions about the
> > Proposal:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >    - How do we plan to test this design to ensure
> > its
> > > > > > >> > correctness?
> > > > > > >> > > Or
> > > > > > >> > > > > > more
> > > > > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’
> > based
> > > > > model
> > > > > > >> is
> > > > > > >> > > > > > functional
> > > > > > >> > > > > > > and
> > > > > > >> > > > > > >    correct given that it is different from the
> > > original
> > > > > RAFT
> > > > > > >> > > > > > implementation
> > > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > We have two planned verifications on the correctness
> > and
> > > > > > >> liveness
> > > > > > >> > of
> > > > > > >> > > > the
> > > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > > > > > >> > > > > >
> > > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > > >> > > > > >
> > > > > > >> > > > > >    - Have we considered any sensible defaults for
> the
> > > > > > >> > configuration,
> > > > > > >> > > > i.e.
> > > > > > >> > > > > > >    all the election timeout, fetch time out, etc.?
> > Or
> > > we
> > > > > > want
> > > > > > >> to
> > > > > > >> > > > leave
> > > > > > >> > > > > > > this to
> > > > > > >> > > > > > >    a later stage when we do the performance
> testing,
> > > > etc.
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > This is a good question, the reason we did not set
> any
> > > > > default
> > > > > > >> > values
> > > > > > >> > > > for
> > > > > > >> > > > > > the timeout configurations is that we think it may
> > take
> > > > some
> > > > > > >> > > > benchmarking
> > > > > > >> > > > > > experiments to get these defaults right. Some
> > high-level
> > > > > > >> principles
> > > > > > >> > > to
> > > > > > >> > > > > > consider: 1) the fetch.timeout should be around the
> > same
> > > > > scale
> > > > > > >> with
> > > > > > >> > > zk
> > > > > > >> > > > > > session timeout, which is now 18 seconds by default
> --
> > > in
> > > > > > >> practice
> > > > > > >> > > > we've
> > > > > > >> > > > > > seen unstable networks having more than 10 secs of
> > > > transient
> > > > > > >> > > > > connectivity,
> > > > > > >> > > > > > 2) the election.timeout, however, should be smaller
> > than
> > > > the
> > > > > > >> fetch
> > > > > > >> > > > > timeout
> > > > > > >> > > > > > as is also suggested as a practical optimization in
> > > > > > literature:
> > > > > > >> > > > > >
> > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > > >> > > > > >
> > > > > > >> > > > > > Some more discussions can be found here:
> > > > > > >> > > > > >
> > > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - Have we considered piggybacking
> > > `BeginQuorumEpoch`
> > > > > with
> > > > > > >> the
> > > > > > >> > `
> > > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing
> something
> > > > > obvious
> > > > > > >> but
> > > > > > >> > I
> > > > > > >> > > am
> > > > > > >> > > > > > just
> > > > > > >> > > > > > >    wondering why don’t we just use the
> `FindQuorum`
> > > and
> > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader
> ->
> > > > other
> > > > > > >> voter
> > > > > > >> > > > > > followers, while FindQuorum / Fetch are sent from
> > > follower
> > > > > to
> > > > > > >> > leader.
> > > > > > >> > > > > > Arguably one can eventually realize the new leader
> and
> > > > epoch
> > > > > > via
> > > > > > >> > > > > gossiping
> > > > > > >> > > > > > FindQuorum, but that could in practice require a
> long
> > > > delay.
> > > > > > >> > Having a
> > > > > > >> > > > > > leader -> other voters request helps the new leader
> > > epoch
> > > > to
> > > > > > be
> > > > > > >> > > > > propagated
> > > > > > >> > > > > > faster under a pull model.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - And about the `FetchQuorumRecords` response
> > > schema,
> > > > > in
> > > > > > >> the
> > > > > > >> > > > > `Records`
> > > > > > >> > > > > > >    field of the response, is it just one record or
> > all
> > > > the
> > > > > > >> > records
> > > > > > >> > > > > > starting
> > > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> > efficient
> > > > if
> > > > > we
> > > > > > >> sent
> > > > > > >> > > all
> > > > > > >> > > > > the
> > > > > > >> > > > > > >    records during the bootstrapping of the
> brokers.
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> > > > > starting
> > > > > > >> > offset
> > > > > > >> > > of
> > > > > > >> > > > > the
> > > > > > >> > > > > > batch of records.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - Regarding the disruptive broker issues, does
> > our
> > > > pull
> > > > > > >> based
> > > > > > >> > > > model
> > > > > > >> > > > > > >    suffer from it? If so, have we considered the
> > > > Pre-Vote
> > > > > > >> stage?
> > > > > > >> > If
> > > > > > >> > > > > not,
> > > > > > >> > > > > > > why?
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > The disruptive broker is stated in the original Raft
> > > paper
> > > > > > >> which is
> > > > > > >> > > the
> > > > > > >> > > > > > result of the push model design. Our analysis showed
> > > that
> > > > > with
> > > > > > >> the
> > > > > > >> > > pull
> > > > > > >> > > > > > model it is no longer an issue.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Thanks a lot for putting this up, and I hope that
> my
> > > > > > questions
> > > > > > >> > can
> > > > > > >> > > be
> > > > > > >> > > > > of
> > > > > > >> > > > > > > some value to make this KIP better.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Hope to hear from you soon!
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Best wishes,
> > > > > > >> > > > > > > Leonard
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > > > >> cmccabe@apache.org
> > > > > > >> > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > Hi Jason,
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > I haven't had a chance to read in detail, but I
> > read
> > > > the
> > > > > > >> > outline
> > > > > > >> > > > and
> > > > > > >> > > > > a
> > > > > > >> > > > > > > few
> > > > > > >> > > > > > > > things jumped out at me.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > First, for every epoch that is 32 bits rather
> than
> > > > 64, I
> > > > > > >> sort
> > > > > > >> > of
> > > > > > >> > > > > wonder
> > > > > > >> > > > > > > if
> > > > > > >> > > > > > > > that's a good long-term choice.  I keep reading
> > > about
> > > > > > stuff
> > > > > > >> > like
> > > > > > >> > > > > this:
> > > > > > >> > > > > > > >
> > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > .
> > > > > > >> > > Obviously,
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > > JIRA is about zxid, which increments much faster
> > > than
> > > > we
> > > > > > >> expect
> > > > > > >> > > > these
> > > > > > >> > > > > > > > leader epochs to, but it would still be good to
> > see
> > > > some
> > > > > > >> rough
> > > > > > >> > > > > > > calculations
> > > > > > >> > > > > > > > about how long 32 bits (or really, 31 bits) will
> > > last
> > > > us
> > > > > > in
> > > > > > >> the
> > > > > > >> > > > cases
> > > > > > >> > > > > > > where
> > > > > > >> > > > > > > > we're using it, and what the space savings we're
> > > > getting
> > > > > > >> really
> > > > > > >> > > is.
> > > > > > >> > > > > It
> > > > > > >> > > > > > > > seems like in most cases the tradeoff may not be
> > > worth
> > > > > it?
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Another thing I've been thinking about is how we
> > do
> > > > > > >> > > > bootstrapping.  I
> > > > > > >> > > > > > > > would prefer to be in a world where formatting a
> > new
> > > > > Kafka
> > > > > > >> node
> > > > > > >> > > > was a
> > > > > > >> > > > > > > first
> > > > > > >> > > > > > > > class operation explicitly initiated by the
> admin,
> > > > > rather
> > > > > > >> than
> > > > > > >> > > > > > something
> > > > > > >> > > > > > > > that happened implicitly when you started up the
> > > > broker
> > > > > > and
> > > > > > >> > > things
> > > > > > >> > > > > > > "looked
> > > > > > >> > > > > > > > blank."
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > The first problem is that things can "look
> blank"
> > > > > > >> accidentally
> > > > > > >> > if
> > > > > > >> > > > the
> > > > > > >> > > > > > > > storage system is having a bad day.  Clearly in
> > the
> > > > > > non-Raft
> > > > > > >> > > world,
> > > > > > >> > > > > > this
> > > > > > >> > > > > > > > leads to data loss if the broker that is
> > (re)started
> > > > > this
> > > > > > >> way
> > > > > > >> > was
> > > > > > >> > > > the
> > > > > > >> > > > > > > > leader for some partitions.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > The second problem is that we have a bit of a
> > > chicken
> > > > > and
> > > > > > >> egg
> > > > > > >> > > > problem
> > > > > > >> > > > > > > with
> > > > > > >> > > > > > > > certain configuration keys.  For example, maybe
> > you
> > > > want
> > > > > > to
> > > > > > >> > > > configure
> > > > > > >> > > > > > > some
> > > > > > >> > > > > > > > connection security settings in your cluster,
> but
> > > you
> > > > > > don't
> > > > > > >> > want
> > > > > > >> > > > them
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > ever be stored in a plaintext config file.  (For
> > > > > example,
> > > > > > >> SCRAM
> > > > > > >> > > > > > > passwords,
> > > > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > > > configuration,
> > > > > > >> but
> > > > > > >> > > > that
> > > > > > >> > > > > > > brings
> > > > > > >> > > > > > > > up the chicken and egg problem.  The broker
> needs
> > to
> > > > be
> > > > > > >> > > configured
> > > > > > >> > > > to
> > > > > > >> > > > > > > know
> > > > > > >> > > > > > > > how to talk to you, but you need to configure it
> > > > before
> > > > > > you
> > > > > > >> can
> > > > > > >> > > > talk
> > > > > > >> > > > > to
> > > > > > >> > > > > > > > it.  Using an external secret manager like Vault
> > is
> > > > one
> > > > > > way
> > > > > > >> to
> > > > > > >> > > > solve
> > > > > > >> > > > > > > this,
> > > > > > >> > > > > > > > but not everyone uses an external secret
> manager.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > quorum.voters seems like a similar configuration
> > > key.
> > > > > In
> > > > > > >> the
> > > > > > >> > > > current
> > > > > > >> > > > > > > KIP,
> > > > > > >> > > > > > > > this is only read if there is no other
> > configuration
> > > > > > >> specifying
> > > > > > >> > > the
> > > > > > >> > > > > > > quorum
> > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> > > > wouldn't
> > > > > > need
> > > > > > >> > this
> > > > > > >> > > > key
> > > > > > >> > > > > > > > because we could assume that there was always
> > quorum
> > > > > > >> > information
> > > > > > >> > > > > stored
> > > > > > >> > > > > > > > locally.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > best,
> > > > > > >> > > > > > > > Colin
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson
> > > wrote:
> > > > > > >> > > > > > > > > Hi All,
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > >> > > > > > > > .
> > > > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > > > ultimately
> > > > > > >> replace
> > > > > > >> > > > > > Zookeeper
> > > > > > >> > > > > > > > > as
> > > > > > >> > > > > > > > > documented in KIP-500. Please take a look and
> > > share
> > > > > your
> > > > > > >> > > > thoughts.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > A few minor notes to set the stage a little
> bit:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > - This KIP does not specify the structure of
> the
> > > > > > messages
> > > > > > >> > used
> > > > > > >> > > to
> > > > > > >> > > > > > > > represent
> > > > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> > > internal
> > > > > API
> > > > > > >> that
> > > > > > >> > > will
> > > > > > >> > > > > be
> > > > > > >> > > > > > > used
> > > > > > >> > > > > > > > > by the controller. Expect these to come in
> later
> > > > > > >> proposals.
> > > > > > >> > > Here
> > > > > > >> > > > we
> > > > > > >> > > > > > are
> > > > > > >> > > > > > > > > primarily concerned with the replication
> > protocol
> > > > and
> > > > > > >> basic
> > > > > > >> > > > > > operational
> > > > > > >> > > > > > > > > mechanics.
> > > > > > >> > > > > > > > > - We expect many details to change as we get
> > > closer
> > > > to
> > > > > > >> > > > integration
> > > > > > >> > > > > > with
> > > > > > >> > > > > > > > > the controller. Any changes we make will be
> made
> > > > > either
> > > > > > as
> > > > > > >> > > > > amendments
> > > > > > >> > > > > > > to
> > > > > > >> > > > > > > > > this KIP or, in the case of larger changes, as
> > new
> > > > > > >> proposals.
> > > > > > >> > > > > > > > > - We have a prototype implementation which I
> > will
> > > > put
> > > > > > >> online
> > > > > > >> > > > within
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > > next week which may help in understanding some
> > > > > details.
> > > > > > It
> > > > > > >> > has
> > > > > > >> > > > > > > diverged a
> > > > > > >> > > > > > > > > little bit from our proposal, so I am taking a
> > > > little
> > > > > > >> time to
> > > > > > >> > > > bring
> > > > > > >> > > > > > it
> > > > > > >> > > > > > > in
> > > > > > >> > > > > > > > > line. I'll post an update to this thread when
> it
> > > is
> > > > > > >> available
> > > > > > >> > > for
> > > > > > >> > > > > > > review.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Finally, I want to mention that this proposal
> > was
> > > > > > drafted
> > > > > > >> by
> > > > > > >> > > > > myself,
> > > > > > >> > > > > > > > Boyang
> > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Thanks,
> > > > > > >> > > > > > > > > Jason
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > --
> > > > > > >> > > > > > > Leonard Ge
> > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > --
> > > > > > >> > > > > > -- Guozhang
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > --
> > > > > > >> > > > -- Guozhang
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > --
> > > > > > >> > -- Guozhang
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Jun,

Thanks for your comments, answering some of them inlined below.

On Thu, Jul 16, 2020 at 3:44 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Jason,
>
> Thanks for the updated KIP. Looks good overall. A few more comments below.
>
> 101. I still don't see a section on bootstrapping related issues. It would
> be useful to document if/how the following is supported.
> 101.1 Currently, we support auto broker id generation. Is this supported
> for bootstrap brokers?
> 101.2 As Colin mentioned, sometimes we may need to load the security
> credentials to be broker before it can be connected to. Could you provide a
> bit more detail on how this will work?
> 101.3 Currently, we use ZK to generate clusterId on a new cluster. With
> Raft, how does every broker generate the same clusterId in a distributed
> way?
>
> 200. It would be useful to document if the various special offsets (log
> start offset, recovery point, HWM, etc) for the Raft log are stored in the
> same existing checkpoint files or not.
> 200.1 Since the Raft log flushes every append, does that allow us to
> recover from a recovery point within the active segment or do we still need
> to scan the full segment including the recovery point? The former can be
> tricky since multiple records can fall into the same disk page and a
> subsequent flush may corrupt a page with previously flushed records.
>

I think we would still document the special offsets for the Raft log in the
existing checkpoint files. I will update the KIP.

I have not thought about optimizing our existing recovery process at the
moment; Raft log flushing on every append may open the door for some
optimization, but on the other hand we are also considering some ways to
defer the every-flush-on-append as well for future works, as suggested in
the KIP docs. So I'd say at the moment we will just keep the recovery logic
as is.


>
> 201. Configurations.
> 201.1 How do the Raft brokers get security related configs for inter broker
> communication? Is that based on the existing
> inter.broker.security.protocol?
>

We have a separate KIP proposal to address broker bootstrapping (actually,
that also includes broker reconfiguration) issues, and I believe Jason
would publish soon. The main idea is that for security types we would still
set it via "security.inter.broker.protocol" and "inter.broker.listener.name",
but the configs would not be allowed to be altered while the broker is
offline. And for the case when a broker is started for the first time,
users would need to set it in "meta.properties" if necessary.


> 201.2 We have quorum.retry.backoff.max.ms and quorum.retry.backoff.ms, but
> only quorum.election.backoff.max.ms. This seems a bit inconsistent.
>
>
The current implementation does not use `quorum.retry.backoff.max.ms`,
since we just used a static backoff logic at the connection layer relying
on `quorum.retry.backoff.ms`. Personally I think it is okay to remove the
former config at the moment since we do not have a strong motivation to use
exponential backoffs.

The election procedure indeed uses a binary exponential backoff mechanism,
but I think we do not need to have a separate `
quorum.election.backoff.base.ms` or something like that -- currently the
base ms is hardcoded -- but I'm open to other thoughts if we believe making
it configurable is also important.


> 202. Metrics:
> 202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs, OutboundQueueTimeMs:
> Are those the same as existing totalTime, requestQueueTime, localTime,
> responseQueueTime? Could we reuse the existing ones with the tag
> request=[request-type]?
>

Yes we can, I'm going to remove those metrics in the KIP.


> 202.2. Could you explain what InboundChannelSize and OutboundChannelSize
> are?
>

Our current implementation uses another channel between the handler to the
RaftClient, that is, upon handling a request the thread does not do
anything but just put it into a new queue which would then be polled by the
RaftClient, and similarly the generated response would be also synced to
the handler threads via a queue as well. And these metrics are for
measuring this queue size.


> 202.3 ElectionLatencyMax/Avg: It seems that both should be windowed?
>
>
You're right, they are implemented as windowed sum / avg, and so are "
*ReplicationLatencyMax/Avg*" actually. I will update the KIP.


> 203. Quorum State: I assume that LeaderId will be kept consistently with
> LeaderEpoch. For example, if a follower transitions to candidate and bumps
> up LeaderEpoch, it will set leaderId to -1 and persist both in the Quorum
> state file. Is that correct?
>
> 204. I was thinking about a corner case when a Raft broker is partitioned
> off. This broker will then be in a continuous loop of bumping up the leader
> epoch, but failing to get enough votes. When the partitioning is removed,
> this broker's high leader epoch will force a leader election. I assume
> other Raft brokers can immediately advance their leader epoch passing the
> already bumped epoch such that leader election won't be delayed. Is that
> right?
>
> 205. In a JBOD setting, could we use the existing tool to move the Raft log
> from one disk to another?
>
> 206. The KIP doesn't mention the local metadata store derived from the Raft
> log. Will that be covered in a separate KIP?
>
> 207. Since this is a critical component. Could we add a section on the
> testing plan for correctness?
>
> 208. Performance. Do we plan to do group commit (e.g. buffer pending
> appends during a flush and then flush all accumulated pending records
> together in the next flush) for better throughput?
>

Yes, that is in our discussion at the moment. Buffering pending appends
have some implicit impacts on some of the raft state transition diagrams
but I think that's doable. I will update the KIP adding a section of
"future works" for this idea.


>
> 209. "the leader can actually defer fsync until it knows "quorum.size - 1"
> has get to a certain entry offset." Why is that "quorum.size - 1" instead
> of the majority of the quorum?
>
>
Yes I meant to say "majority - 1" (minus one for the leader itself). I will
update the KIP.


> Thanks,
>
> Jun
>
> On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi All,
> >
> > Just a quick update on the proposal. We have decided to move quorum
> > reassignment to a separate KIP:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> > .
> > The way this ties into cluster bootstrapping is complicated, so we felt
> we
> > needed a bit more time for validation. That leaves the core of this
> > proposal as quorum-based replication. If there are no further comments,
> we
> > will plan to start a vote later this week.
> >
> > Thanks,
> > Jason
> >
> > On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > @Jun Rao <ju...@gmail.com>
> > >
> > > Regarding your comment about log compaction. After some deep-diving
> into
> > > this we've decided to propose a new snapshot-based log cleaning
> mechanism
> > > which would be used to replace the current compaction mechanism for
> this
> > > meta log. A new KIP will be proposed specifically for this idea.
> > >
> > > All,
> > >
> > > I've updated the KIP wiki a bit updating one config "
> > > election.jitter.max.ms"
> > > to "election.backoff.max.ms" to make it more clear about the usage:
> the
> > > configured value will be the upper bound of the binary exponential
> > backoff
> > > time after a failed election, before starting a new one.
> > >
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <
> reluctanthero104@gmail.com>
> > > wrote:
> > >
> > > > Thanks for the suggestions Guozhang.
> > > >
> > > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hello Boyang,
> > > > >
> > > > > Thanks for the updated information. A few questions here:
> > > > >
> > > > > 1) Should the quorum-file also update to support multi-raft?
> > > > >
> > > > > I'm neutral about this, as we don't know yet how the multi-raft
> > modules
> > > > would behave. If
> > > > we have different threads operating different raft groups,
> > consolidating
> > > > the `checkpoint` files seems
> > > > not reasonable. We could always add `multi-quorum-file` later if
> > > possible.
> > > >
> > > > 2) In the previous proposal, there's fields in the FetchQuorumRecords
> > > like
> > > > > latestDirtyOffset, is that dropped intentionally?
> > > > >
> > > > > I dropped the latestDirtyOffset since it is associated with the log
> > > > compaction discussion. This is beyond this KIP scope and we could
> > > > potentially get a separate KIP to talk about it.
> > > >
> > > >
> > > > > 3) I think we also need to elaborate a bit more details regarding
> > when
> > > to
> > > > > send metadata request and discover-brokers; currently we only
> > discussed
> > > > > during bootstrap how these requests would be sent. I think the
> > > following
> > > > > scenarios would also need these requests
> > > > >
> > > > > 3.a) As long as a broker does not know the current quorum
> (including
> > > the
> > > > > leader and the voters), it should continue periodically ask other
> > > brokers
> > > > > via "metadata.
> > > > > 3.b) As long as a broker does not know all the current quorum
> voter's
> > > > > connections, it should continue periodically ask other brokers via
> > > > > "discover-brokers".
> > > > > 3.c) When the leader's fetch timeout elapsed, it should send
> metadata
> > > > > request.
> > > > >
> > > > > Make sense, will add to the KIP.
> > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey all,
> > > > > >
> > > > > > follow-up on the previous email, we made some more updates:
> > > > > >
> > > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > > > > multi-raft.
> > > > > >
> > > > > > 2. We add observer status into the DescribeQuorumResponse as we
> see
> > > it
> > > > > is a
> > > > > > low hanging fruit which is very useful for user debugging and
> > > > > reassignment.
> > > > > >
> > > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which
> > is
> > > > > purely
> > > > > > in charge of discovering broker connections in a gossip manner.
> The
> > > > > quorum
> > > > > > leader discovery is piggy-back on the Metadata RPC for the topic
> > > > > partition
> > > > > > leader, which in our case is the single metadata partition for
> the
> > > > > version
> > > > > > one.
> > > > > >
> > > > > > Let me know if you have any questions.
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > > reluctanthero104@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey all,
> > > > > > >
> > > > > > > Thanks for the great discussions so far. I'm posting some KIP
> > > updates
> > > > > > from
> > > > > > > our working group discussion:
> > > > > > >
> > > > > > > 1. We will be changing the core RPCs from single-raft API to
> > > > > multi-raft.
> > > > > > > This means all protocols will be "batch" in the first version,
> > but
> > > > the
> > > > > > KIP
> > > > > > > itself only illustrates the design for a single metadata topic
> > > > > partition.
> > > > > > > The reason is to "keep the door open" for future extensions of
> > this
> > > > > piece
> > > > > > > of module such as a sharded controller or general quorum based
> > > topic
> > > > > > > replication, beyond the current Kafka replication protocol.
> > > > > > >
> > > > > > > 2. We will piggy-back on the current Kafka Fetch API instead of
> > > > > inventing
> > > > > > > a new FetchQuorumRecords RPC. The motivation is about the same
> as
> > > #1
> > > > as
> > > > > > > well as making the integration work easier, instead of letting
> > two
> > > > > > similar
> > > > > > > RPCs diverge.
> > > > > > >
> > > > > > > 3. In the EndQuorumEpoch protocol, instead of only sending the
> > > > request
> > > > > to
> > > > > > > the most caught-up voter, we shall broadcast the information to
> > all
> > > > > > voters,
> > > > > > > with a sorted voter list in descending order of their
> > corresponding
> > > > > > > replicated offset. In this way, the top voter will become a
> > > candidate
> > > > > > > immediately, while the other voters shall wait for an
> exponential
> > > > > > back-off
> > > > > > > to trigger elections, which helps ensure the top voter gets
> > > elected,
> > > > > and
> > > > > > > the election eventually happens when the top voter is not
> > > responsive.
> > > > > > >
> > > > > > > Please see the updated KIP and post any questions or concerns
> on
> > > the
> > > > > > > mailing thread.
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > >
> > > > > > >> Hi, Guozhang and Jason,
> > > > > > >>
> > > > > > >> Thanks for the reply. A couple of more replies.
> > > > > > >>
> > > > > > >> 102. Still not sure about this. How is the tombstone issue
> > > addressed
> > > > > in
> > > > > > >> the
> > > > > > >> non-voter and the observer.  They can die at any point and
> > restart
> > > > at
> > > > > an
> > > > > > >> arbitrary later time, and the advancing of the firstDirty
> offset
> > > and
> > > > > the
> > > > > > >> removal of the tombstone can happen independently.
> > > > > > >>
> > > > > > >> 106. I agree that it would be less confusing if we used
> "epoch"
> > > > > instead
> > > > > > of
> > > > > > >> "leader epoch" consistently.
> > > > > > >>
> > > > > > >> Jun
> > > > > > >>
> > > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > > >> >
> > > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > >> >
> > > > > > >> > > Hi, Guozhang,
> > > > > > >> > >
> > > > > > >> > > Thanks for the reply. A few more replies inlined below.
> > > > > > >> > >
> > > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Hello Jun,
> > > > > > >> > > >
> > > > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > > > >> > > >
> > > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > > >> > > > > 101.1 Currently, we support auto broker id generation.
> > Is
> > > > this
> > > > > > >> > > supported
> > > > > > >> > > > > for bootstrap brokers?
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > The vote ids would just be the broker ids.
> > > "bootstrap.servers"
> > > > > > >> would be
> > > > > > >> > > > similar to what client configs have today, where
> > > > "quorum.voters"
> > > > > > >> would
> > > > > > >> > be
> > > > > > >> > > > pre-defined config values.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > My question was on the auto generated broker id.
> Currently,
> > > the
> > > > > > broker
> > > > > > >> > can
> > > > > > >> > > choose to have its broker Id auto generated. The
> generation
> > is
> > > > > done
> > > > > > >> > through
> > > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how
> > the
> > > > > > broker
> > > > > > >> id
> > > > > > >> > is
> > > > > > >> > > auto generated. "quorum.voters" also can't be set
> statically
> > > if
> > > > > > broker
> > > > > > >> > ids
> > > > > > >> > > are auto generated.
> > > > > > >> > >
> > > > > > >> > > Jason has explained some ideas that we've discussed so
> far,
> > > the
> > > > > > >> reason we
> > > > > > >> > intentional did not include them so far is that we feel it
> is
> > > > > out-side
> > > > > > >> the
> > > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> > > > definitely
> > > > > > >> > address them though.
> > > > > > >> >
> > > > > > >> > On the high-level, our belief is that "joining a quorum" and
> > > > > "joining
> > > > > > >> (or
> > > > > > >> > more specifically, registering brokers in) the cluster"
> would
> > be
> > > > > > >> > de-coupled a bit, where the former should be completed
> before
> > we
> > > > do
> > > > > > the
> > > > > > >> > latter. More specifically, assuming the quorum is already up
> > and
> > > > > > >> running,
> > > > > > >> > after the newly started broker found the leader of the
> quorum
> > it
> > > > can
> > > > > > >> send a
> > > > > > >> > specific RegisterBroker request including its listener /
> > > protocol
> > > > /
> > > > > > etc,
> > > > > > >> > and upon handling it the leader can send back the uniquely
> > > > generated
> > > > > > >> broker
> > > > > > >> > id to the new broker, while also executing the
> > "startNewBroker"
> > > > > > >> callback as
> > > > > > >> > the controller.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >
> > > > > > >> > > > > 102. Log compaction. One weak spot of log compaction
> is
> > > for
> > > > > the
> > > > > > >> > > consumer
> > > > > > >> > > > to
> > > > > > >> > > > > deal with deletes. When a key is deleted, it's
> retained
> > > as a
> > > > > > >> > tombstone
> > > > > > >> > > > > first and then physically removed. If a client misses
> > the
> > > > > > >> tombstone
> > > > > > >> > > > > (because it's physically removed), it may not be able
> to
> > > > > update
> > > > > > >> its
> > > > > > >> > > > > metadata properly. The way we solve this in Kafka is
> > based
> > > > on
> > > > > a
> > > > > > >> > > > > configuration (log.cleaner.delete.retention.ms) and
> we
> > > > > expect a
> > > > > > >> > > consumer
> > > > > > >> > > > > having seen an old key to finish reading the deletion
> > > > > tombstone
> > > > > > >> > within
> > > > > > >> > > > that
> > > > > > >> > > > > time. There is no strong guarantee for that since a
> > broker
> > > > > could
> > > > > > >> be
> > > > > > >> > > down
> > > > > > >> > > > > for a long time. It would be better if we can have a
> > more
> > > > > > reliable
> > > > > > >> > way
> > > > > > >> > > of
> > > > > > >> > > > > dealing with deletes.
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > We propose to capture this in the "FirstDirtyOffset"
> field
> > > of
> > > > > the
> > > > > > >> > quorum
> > > > > > >> > > > record fetch response: the offset is the maximum offset
> > that
> > > > log
> > > > > > >> > > compaction
> > > > > > >> > > > has reached up to. If the follower has fetched beyond
> this
> > > > > offset
> > > > > > it
> > > > > > >> > > means
> > > > > > >> > > > itself is safe hence it has seen all records up to that
> > > > offset.
> > > > > On
> > > > > > >> > > getting
> > > > > > >> > > > the response, the follower can then decide if its end
> > offset
> > > > > > >> actually
> > > > > > >> > > below
> > > > > > >> > > > that dirty offset (and hence may miss some tombstones).
> If
> > > > > that's
> > > > > > >> the
> > > > > > >> > > case:
> > > > > > >> > > >
> > > > > > >> > > > 1) Naively, it could re-bootstrap metadata log from the
> > very
> > > > > > >> beginning
> > > > > > >> > to
> > > > > > >> > > > catch up.
> > > > > > >> > > > 2) During that time, it would refrain itself from
> > answering
> > > > > > >> > > MetadataRequest
> > > > > > >> > > > from any clients.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> > > addresses
> > > > > the
> > > > > > >> > issue.
> > > > > > >> > > Currently, the deletion tombstone is not removed
> immediately
> > > > > after a
> > > > > > >> > round
> > > > > > >> > > of cleaning. It's removed after a delay in a subsequent
> > round
> > > of
> > > > > > >> > cleaning.
> > > > > > >> > > Consider an example where a key insertion is at offset 200
> > > and a
> > > > > > >> deletion
> > > > > > >> > > tombstone of the key is at 400. Initially,
> FirstDirtyOffset
> > is
> > > > at
> > > > > > >> 300. A
> > > > > > >> > > follower/observer fetches from offset 0  and fetches the
> key
> > > at
> > > > > > offset
> > > > > > >> > 200.
> > > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at
> 500
> > > and
> > > > > the
> > > > > > >> > > tombstone at 400 is physically removed. The
> > follower/observer
> > > > > > >> continues
> > > > > > >> > the
> > > > > > >> > > fetch, but misses offset 400. It catches all the way to
> > > > > > >> FirstDirtyOffset
> > > > > > >> > > and declares its metadata as ready. However, its metadata
> > > could
> > > > be
> > > > > > >> stale
> > > > > > >> > > since it actually misses the deletion of the key.
> > > > > > >> > >
> > > > > > >> > > Yeah good question, I should have put more details in my
> > > > > explanation
> > > > > > >> :)
> > > > > > >> >
> > > > > > >> > The idea is that we will adjust the log compaction for this
> > raft
> > > > > based
> > > > > > >> > metadata log: before more details to be explained, since we
> > have
> > > > two
> > > > > > >> types
> > > > > > >> > of "watermarks" here, whereas in Kafka the watermark
> indicates
> > > > where
> > > > > > >> every
> > > > > > >> > replica have replicated up to and in Raft the watermark
> > > indicates
> > > > > > where
> > > > > > >> the
> > > > > > >> > majority of replicas (here only indicating voters of the
> > quorum,
> > > > not
> > > > > > >> > counting observers) have replicated up to, let's call them
> > Kafka
> > > > > > >> watermark
> > > > > > >> > and Raft watermark. For this special log, we would maintain
> > both
> > > > > > >> > watermarks.
> > > > > > >> >
> > > > > > >> > When log compacting on the leader, we would only compact up
> to
> > > the
> > > > > > Kafka
> > > > > > >> > watermark, i.e. if there is at least one voter who have not
> > > > > replicated
> > > > > > >> an
> > > > > > >> > entry, it would not be compacted. The "dirty-offset" is the
> > > offset
> > > > > > that
> > > > > > >> > we've compacted up to and is communicated to other voters,
> and
> > > the
> > > > > > other
> > > > > > >> > voters would also compact up to this value --- i.e. the
> > > difference
> > > > > > here
> > > > > > >> is
> > > > > > >> > that instead of letting each replica doing log compaction
> > > > > > independently,
> > > > > > >> > we'll have the leader to decide upon which offset to compact
> > to,
> > > > and
> > > > > > >> > propagate this value to others to follow, in a more
> > coordinated
> > > > > > manner.
> > > > > > >> > Also note when there are new voters joining the quorum who
> has
> > > not
> > > > > > >> > replicated up to the dirty-offset, of because of other
> issues
> > > they
> > > > > > >> > truncated their logs to below the dirty-offset, they'd have
> to
> > > > > > >> re-bootstrap
> > > > > > >> > from the beginning, and during this period of time the
> leader
> > > > > learned
> > > > > > >> about
> > > > > > >> > this lagging voter would not advance the watermark (also it
> > > would
> > > > > not
> > > > > > >> > decrement it), and hence not compacting either, until the
> > > voter(s)
> > > > > has
> > > > > > >> > caught up to that dirty-offset.
> > > > > > >> >
> > > > > > >> > So back to your example above, before the bootstrap voter
> gets
> > > to
> > > > > 300
> > > > > > no
> > > > > > >> > log compaction would happen on the leader; and until later
> > when
> > > > the
> > > > > > >> voter
> > > > > > >> > have got to beyond 400 and hence replicated that tombstone,
> > the
> > > > log
> > > > > > >> > compaction would possibly get to that tombstone and remove
> it.
> > > Say
> > > > > > >> later it
> > > > > > >> > the leader's log compaction reaches 500, it can send this
> back
> > > to
> > > > > the
> > > > > > >> voter
> > > > > > >> > who can then also compact locally up to 500.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we need
> > the
> > > > > epoch
> > > > > > >> > > > > corresponding to VotedId? Over time, the same broker
> Id
> > > > could
> > > > > be
> > > > > > >> > voted
> > > > > > >> > > in
> > > > > > >> > > > > different generations with different epoch.
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > > "LeaderEpoch"
> > > > > > >> field
> > > > > > >> > in
> > > > > > >> > > > that file is corresponding to the "latest known leader
> > > epoch",
> > > > > not
> > > > > > >> the
> > > > > > >> > > > "current leader epoch". For example, if the current
> epoch
> > is
> > > > N,
> > > > > > and
> > > > > > >> > then
> > > > > > >> > > a
> > > > > > >> > > > vote-request with epoch N+1 is received and the voter
> > > granted
> > > > > the
> > > > > > >> vote
> > > > > > >> > > for
> > > > > > >> > > > it, then it means for this voter it knows the "latest
> > epoch"
> > > > is
> > > > > N
> > > > > > +
> > > > > > >> 1
> > > > > > >> > > > although it is unknown if that sending candidate will
> > indeed
> > > > > > become
> > > > > > >> the
> > > > > > >> > > new
> > > > > > >> > > > leader (which would only be notified via begin-quorum
> > > > request).
> > > > > > >> > However,
> > > > > > >> > > > when persisting the quorum state, we would encode
> > > leader-epoch
> > > > > to
> > > > > > >> N+1,
> > > > > > >> > > > while the leaderId to be the older leader.
> > > > > > >> > > >
> > > > > > >> > > > But now thinking about this a bit more, I feel we should
> > use
> > > > two
> > > > > > >> > separate
> > > > > > >> > > > epochs, one for the "lates known" and one for the
> > "current"
> > > to
> > > > > > pair
> > > > > > >> > with
> > > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch before
> > the
> > > > new
> > > > > > >> leader
> > > > > > >> > > is actually elected, right.
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> > FetchQuorumRecords
> > > > API
> > > > > to
> > > > > > >> > > indicate
> > > > > > >> > > > > that the follower has fetched from an invalid offset
> and
> > > > > should
> > > > > > >> > > truncate
> > > > > > >> > > > to
> > > > > > >> > > > > the offset/epoch indicated in the response." Observers
> > > can't
> > > > > > >> truncate
> > > > > > >> > > > their
> > > > > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > I'm not sure if I understand your question? Observers
> > should
> > > > > still
> > > > > > >> be
> > > > > > >> > > able
> > > > > > >> > > > to truncate their logs as well.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > Hmm, I thought only the quorum nodes have local logs and
> > > > observers
> > > > > > >> don't?
> > > > > > >> > >
> > > > > > >> > > > 107. "The leader will continue sending BeginQuorumEpoch
> to
> > > > each
> > > > > > >> known
> > > > > > >> > > > voter
> > > > > > >> > > > > until it has received its endorsement." If a voter is
> > down
> > > > > for a
> > > > > > >> long
> > > > > > >> > > > time,
> > > > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> > > overhead.
> > > > > > >> > Similarly,
> > > > > > >> > > > if a
> > > > > > >> > > > > follower stops sending FetchQuorumRecords, does the
> > leader
> > > > > keep
> > > > > > >> > sending
> > > > > > >> > > > > BeginQuorumEpoch?
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > > > > >> > begin-quorum-epoch
> > > > > > >> > > > request is for voters to quickly get the new leader
> > > > information;
> > > > > > >> > however
> > > > > > >> > > > even if they do not get them they can still eventually
> > learn
> > > > > about
> > > > > > >> that
> > > > > > >> > > > from others via gossiping FindQuorum. I think we can
> > adjust
> > > > the
> > > > > > >> logic
> > > > > > >> > to
> > > > > > >> > > > e.g. exponential back-off or with a limited num.retries.
> > > > > > >> > > >
> > > > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > > > >> FetchQuorumRecords
> > > > > > >> > > > already, it means that follower already knows that the
> > > broker
> > > > is
> > > > > > the
> > > > > > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch;
> > > > however
> > > > > > it
> > > > > > >> is
> > > > > > >> > > > possible that after a follower sends FetchQuorumRecords
> > > > already,
> > > > > > >> > suddenly
> > > > > > >> > > > it stops send it (possibly because it learned about a
> > higher
> > > > > epoch
> > > > > > >> > > leader),
> > > > > > >> > > > and hence this broker may be a "zombie" leader and we
> > > propose
> > > > to
> > > > > > use
> > > > > > >> > the
> > > > > > >> > > > fetch.timeout to let the leader to try to verify if it
> has
> > > > > already
> > > > > > >> been
> > > > > > >> > > > stale.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > It just seems that we should handle these two cases in a
> > > > > consistent
> > > > > > >> way?
> > > > > > >> > >
> > > > > > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords
> > > from a
> > > > > > >> follower
> > > > > > >> > could mean that we no longer needs to send BeginQuorumEpoch
> > > > anymore
> > > > > > ---
> > > > > > >> and
> > > > > > >> > it is already part of our current implementations in
> > > > > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > Thanks,
> > > > > > >> > >
> > > > > > >> > > Jun
> > > > > > >> > >
> > > > > > >> > > >
> > > > > > >> > > > >
> > > > > > >> > > > > Jun
> > > > > > >> > > > >
> > > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > >> >
> > > > > > >> > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hello Leonard,
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks for your comments, I'm relying in line below:
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > > > > > >> > wge@confluent.io>
> > > > > > >> > > > > > wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Hi Kafka developers,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > It's great to see this proposal and it took me
> some
> > > time
> > > > > to
> > > > > > >> > finish
> > > > > > >> > > > > > reading
> > > > > > >> > > > > > > it.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > And I have the following questions about the
> > Proposal:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >    - How do we plan to test this design to ensure
> > its
> > > > > > >> > correctness?
> > > > > > >> > > Or
> > > > > > >> > > > > > more
> > > > > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’
> > based
> > > > > model
> > > > > > >> is
> > > > > > >> > > > > > functional
> > > > > > >> > > > > > > and
> > > > > > >> > > > > > >    correct given that it is different from the
> > > original
> > > > > RAFT
> > > > > > >> > > > > > implementation
> > > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > We have two planned verifications on the correctness
> > and
> > > > > > >> liveness
> > > > > > >> > of
> > > > > > >> > > > the
> > > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > > > > > >> > > > > >
> > > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > > >> > > > > >
> > > > > > >> > > > > >    - Have we considered any sensible defaults for
> the
> > > > > > >> > configuration,
> > > > > > >> > > > i.e.
> > > > > > >> > > > > > >    all the election timeout, fetch time out, etc.?
> > Or
> > > we
> > > > > > want
> > > > > > >> to
> > > > > > >> > > > leave
> > > > > > >> > > > > > > this to
> > > > > > >> > > > > > >    a later stage when we do the performance
> testing,
> > > > etc.
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > This is a good question, the reason we did not set
> any
> > > > > default
> > > > > > >> > values
> > > > > > >> > > > for
> > > > > > >> > > > > > the timeout configurations is that we think it may
> > take
> > > > some
> > > > > > >> > > > benchmarking
> > > > > > >> > > > > > experiments to get these defaults right. Some
> > high-level
> > > > > > >> principles
> > > > > > >> > > to
> > > > > > >> > > > > > consider: 1) the fetch.timeout should be around the
> > same
> > > > > scale
> > > > > > >> with
> > > > > > >> > > zk
> > > > > > >> > > > > > session timeout, which is now 18 seconds by default
> --
> > > in
> > > > > > >> practice
> > > > > > >> > > > we've
> > > > > > >> > > > > > seen unstable networks having more than 10 secs of
> > > > transient
> > > > > > >> > > > > connectivity,
> > > > > > >> > > > > > 2) the election.timeout, however, should be smaller
> > than
> > > > the
> > > > > > >> fetch
> > > > > > >> > > > > timeout
> > > > > > >> > > > > > as is also suggested as a practical optimization in
> > > > > > literature:
> > > > > > >> > > > > >
> > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > > >> > > > > >
> > > > > > >> > > > > > Some more discussions can be found here:
> > > > > > >> > > > > >
> > > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - Have we considered piggybacking
> > > `BeginQuorumEpoch`
> > > > > with
> > > > > > >> the
> > > > > > >> > `
> > > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing
> something
> > > > > obvious
> > > > > > >> but
> > > > > > >> > I
> > > > > > >> > > am
> > > > > > >> > > > > > just
> > > > > > >> > > > > > >    wondering why don’t we just use the
> `FindQuorum`
> > > and
> > > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader
> ->
> > > > other
> > > > > > >> voter
> > > > > > >> > > > > > followers, while FindQuorum / Fetch are sent from
> > > follower
> > > > > to
> > > > > > >> > leader.
> > > > > > >> > > > > > Arguably one can eventually realize the new leader
> and
> > > > epoch
> > > > > > via
> > > > > > >> > > > > gossiping
> > > > > > >> > > > > > FindQuorum, but that could in practice require a
> long
> > > > delay.
> > > > > > >> > Having a
> > > > > > >> > > > > > leader -> other voters request helps the new leader
> > > epoch
> > > > to
> > > > > > be
> > > > > > >> > > > > propagated
> > > > > > >> > > > > > faster under a pull model.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - And about the `FetchQuorumRecords` response
> > > schema,
> > > > > in
> > > > > > >> the
> > > > > > >> > > > > `Records`
> > > > > > >> > > > > > >    field of the response, is it just one record or
> > all
> > > > the
> > > > > > >> > records
> > > > > > >> > > > > > starting
> > > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> > efficient
> > > > if
> > > > > we
> > > > > > >> sent
> > > > > > >> > > all
> > > > > > >> > > > > the
> > > > > > >> > > > > > >    records during the bootstrapping of the
> brokers.
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> > > > > starting
> > > > > > >> > offset
> > > > > > >> > > of
> > > > > > >> > > > > the
> > > > > > >> > > > > > batch of records.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > >    - Regarding the disruptive broker issues, does
> > our
> > > > pull
> > > > > > >> based
> > > > > > >> > > > model
> > > > > > >> > > > > > >    suffer from it? If so, have we considered the
> > > > Pre-Vote
> > > > > > >> stage?
> > > > > > >> > If
> > > > > > >> > > > > not,
> > > > > > >> > > > > > > why?
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > The disruptive broker is stated in the original Raft
> > > paper
> > > > > > >> which is
> > > > > > >> > > the
> > > > > > >> > > > > > result of the push model design. Our analysis showed
> > > that
> > > > > with
> > > > > > >> the
> > > > > > >> > > pull
> > > > > > >> > > > > > model it is no longer an issue.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Thanks a lot for putting this up, and I hope that
> my
> > > > > > questions
> > > > > > >> > can
> > > > > > >> > > be
> > > > > > >> > > > > of
> > > > > > >> > > > > > > some value to make this KIP better.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Hope to hear from you soon!
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Best wishes,
> > > > > > >> > > > > > > Leonard
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > > > >> cmccabe@apache.org
> > > > > > >> > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > Hi Jason,
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > I haven't had a chance to read in detail, but I
> > read
> > > > the
> > > > > > >> > outline
> > > > > > >> > > > and
> > > > > > >> > > > > a
> > > > > > >> > > > > > > few
> > > > > > >> > > > > > > > things jumped out at me.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > First, for every epoch that is 32 bits rather
> than
> > > > 64, I
> > > > > > >> sort
> > > > > > >> > of
> > > > > > >> > > > > wonder
> > > > > > >> > > > > > > if
> > > > > > >> > > > > > > > that's a good long-term choice.  I keep reading
> > > about
> > > > > > stuff
> > > > > > >> > like
> > > > > > >> > > > > this:
> > > > > > >> > > > > > > >
> > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > > .
> > > > > > >> > > Obviously,
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > > JIRA is about zxid, which increments much faster
> > > than
> > > > we
> > > > > > >> expect
> > > > > > >> > > > these
> > > > > > >> > > > > > > > leader epochs to, but it would still be good to
> > see
> > > > some
> > > > > > >> rough
> > > > > > >> > > > > > > calculations
> > > > > > >> > > > > > > > about how long 32 bits (or really, 31 bits) will
> > > last
> > > > us
> > > > > > in
> > > > > > >> the
> > > > > > >> > > > cases
> > > > > > >> > > > > > > where
> > > > > > >> > > > > > > > we're using it, and what the space savings we're
> > > > getting
> > > > > > >> really
> > > > > > >> > > is.
> > > > > > >> > > > > It
> > > > > > >> > > > > > > > seems like in most cases the tradeoff may not be
> > > worth
> > > > > it?
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Another thing I've been thinking about is how we
> > do
> > > > > > >> > > > bootstrapping.  I
> > > > > > >> > > > > > > > would prefer to be in a world where formatting a
> > new
> > > > > Kafka
> > > > > > >> node
> > > > > > >> > > > was a
> > > > > > >> > > > > > > first
> > > > > > >> > > > > > > > class operation explicitly initiated by the
> admin,
> > > > > rather
> > > > > > >> than
> > > > > > >> > > > > > something
> > > > > > >> > > > > > > > that happened implicitly when you started up the
> > > > broker
> > > > > > and
> > > > > > >> > > things
> > > > > > >> > > > > > > "looked
> > > > > > >> > > > > > > > blank."
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > The first problem is that things can "look
> blank"
> > > > > > >> accidentally
> > > > > > >> > if
> > > > > > >> > > > the
> > > > > > >> > > > > > > > storage system is having a bad day.  Clearly in
> > the
> > > > > > non-Raft
> > > > > > >> > > world,
> > > > > > >> > > > > > this
> > > > > > >> > > > > > > > leads to data loss if the broker that is
> > (re)started
> > > > > this
> > > > > > >> way
> > > > > > >> > was
> > > > > > >> > > > the
> > > > > > >> > > > > > > > leader for some partitions.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > The second problem is that we have a bit of a
> > > chicken
> > > > > and
> > > > > > >> egg
> > > > > > >> > > > problem
> > > > > > >> > > > > > > with
> > > > > > >> > > > > > > > certain configuration keys.  For example, maybe
> > you
> > > > want
> > > > > > to
> > > > > > >> > > > configure
> > > > > > >> > > > > > > some
> > > > > > >> > > > > > > > connection security settings in your cluster,
> but
> > > you
> > > > > > don't
> > > > > > >> > want
> > > > > > >> > > > them
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > ever be stored in a plaintext config file.  (For
> > > > > example,
> > > > > > >> SCRAM
> > > > > > >> > > > > > > passwords,
> > > > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > > > configuration,
> > > > > > >> but
> > > > > > >> > > > that
> > > > > > >> > > > > > > brings
> > > > > > >> > > > > > > > up the chicken and egg problem.  The broker
> needs
> > to
> > > > be
> > > > > > >> > > configured
> > > > > > >> > > > to
> > > > > > >> > > > > > > know
> > > > > > >> > > > > > > > how to talk to you, but you need to configure it
> > > > before
> > > > > > you
> > > > > > >> can
> > > > > > >> > > > talk
> > > > > > >> > > > > to
> > > > > > >> > > > > > > > it.  Using an external secret manager like Vault
> > is
> > > > one
> > > > > > way
> > > > > > >> to
> > > > > > >> > > > solve
> > > > > > >> > > > > > > this,
> > > > > > >> > > > > > > > but not everyone uses an external secret
> manager.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > quorum.voters seems like a similar configuration
> > > key.
> > > > > In
> > > > > > >> the
> > > > > > >> > > > current
> > > > > > >> > > > > > > KIP,
> > > > > > >> > > > > > > > this is only read if there is no other
> > configuration
> > > > > > >> specifying
> > > > > > >> > > the
> > > > > > >> > > > > > > quorum
> > > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> > > > wouldn't
> > > > > > need
> > > > > > >> > this
> > > > > > >> > > > key
> > > > > > >> > > > > > > > because we could assume that there was always
> > quorum
> > > > > > >> > information
> > > > > > >> > > > > stored
> > > > > > >> > > > > > > > locally.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > best,
> > > > > > >> > > > > > > > Colin
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson
> > > wrote:
> > > > > > >> > > > > > > > > Hi All,
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > >> > > > > > > > .
> > > > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > > > ultimately
> > > > > > >> replace
> > > > > > >> > > > > > Zookeeper
> > > > > > >> > > > > > > > > as
> > > > > > >> > > > > > > > > documented in KIP-500. Please take a look and
> > > share
> > > > > your
> > > > > > >> > > > thoughts.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > A few minor notes to set the stage a little
> bit:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > - This KIP does not specify the structure of
> the
> > > > > > messages
> > > > > > >> > used
> > > > > > >> > > to
> > > > > > >> > > > > > > > represent
> > > > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> > > internal
> > > > > API
> > > > > > >> that
> > > > > > >> > > will
> > > > > > >> > > > > be
> > > > > > >> > > > > > > used
> > > > > > >> > > > > > > > > by the controller. Expect these to come in
> later
> > > > > > >> proposals.
> > > > > > >> > > Here
> > > > > > >> > > > we
> > > > > > >> > > > > > are
> > > > > > >> > > > > > > > > primarily concerned with the replication
> > protocol
> > > > and
> > > > > > >> basic
> > > > > > >> > > > > > operational
> > > > > > >> > > > > > > > > mechanics.
> > > > > > >> > > > > > > > > - We expect many details to change as we get
> > > closer
> > > > to
> > > > > > >> > > > integration
> > > > > > >> > > > > > with
> > > > > > >> > > > > > > > > the controller. Any changes we make will be
> made
> > > > > either
> > > > > > as
> > > > > > >> > > > > amendments
> > > > > > >> > > > > > > to
> > > > > > >> > > > > > > > > this KIP or, in the case of larger changes, as
> > new
> > > > > > >> proposals.
> > > > > > >> > > > > > > > > - We have a prototype implementation which I
> > will
> > > > put
> > > > > > >> online
> > > > > > >> > > > within
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > > next week which may help in understanding some
> > > > > details.
> > > > > > It
> > > > > > >> > has
> > > > > > >> > > > > > > diverged a
> > > > > > >> > > > > > > > > little bit from our proposal, so I am taking a
> > > > little
> > > > > > >> time to
> > > > > > >> > > > bring
> > > > > > >> > > > > > it
> > > > > > >> > > > > > > in
> > > > > > >> > > > > > > > > line. I'll post an update to this thread when
> it
> > > is
> > > > > > >> available
> > > > > > >> > > for
> > > > > > >> > > > > > > review.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Finally, I want to mention that this proposal
> > was
> > > > > > drafted
> > > > > > >> by
> > > > > > >> > > > > myself,
> > > > > > >> > > > > > > > Boyang
> > > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Thanks,
> > > > > > >> > > > > > > > > Jason
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > --
> > > > > > >> > > > > > > Leonard Ge
> > > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > --
> > > > > > >> > > > > > -- Guozhang
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > --
> > > > > > >> > > > -- Guozhang
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > --
> > > > > > >> > -- Guozhang
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jun Rao <ju...@confluent.io>.
Hi, Jason,

Thanks for the updated KIP. Looks good overall. A few more comments below.

101. I still don't see a section on bootstrapping related issues. It would
be useful to document if/how the following is supported.
101.1 Currently, we support auto broker id generation. Is this supported
for bootstrap brokers?
101.2 As Colin mentioned, sometimes we may need to load the security
credentials to be broker before it can be connected to. Could you provide a
bit more detail on how this will work?
101.3 Currently, we use ZK to generate clusterId on a new cluster. With
Raft, how does every broker generate the same clusterId in a distributed
way?

200. It would be useful to document if the various special offsets (log
start offset, recovery point, HWM, etc) for the Raft log are stored in the
same existing checkpoint files or not.
200.1 Since the Raft log flushes every append, does that allow us to
recover from a recovery point within the active segment or do we still need
to scan the full segment including the recovery point? The former can be
tricky since multiple records can fall into the same disk page and a
subsequent flush may corrupt a page with previously flushed records.

201. Configurations.
201.1 How do the Raft brokers get security related configs for inter broker
communication? Is that based on the existing inter.broker.security.protocol?
201.2 We have quorum.retry.backoff.max.ms and quorum.retry.backoff.ms, but
only quorum.election.backoff.max.ms. This seems a bit inconsistent.

202. Metrics:
202.1 TotalTimeMs, InboundQueueTimeMs, HandleTimeMs, OutboundQueueTimeMs:
Are those the same as existing totalTime, requestQueueTime, localTime,
responseQueueTime? Could we reuse the existing ones with the tag
request=[request-type]?
202.2. Could you explain what InboundChannelSize and OutboundChannelSize
are?
202.3 ElectionLatencyMax/Avg: It seems that both should be windowed?

203. Quorum State: I assume that LeaderId will be kept consistently with
LeaderEpoch. For example, if a follower transitions to candidate and bumps
up LeaderEpoch, it will set leaderId to -1 and persist both in the Quorum
state file. Is that correct?

204. I was thinking about a corner case when a Raft broker is partitioned
off. This broker will then be in a continuous loop of bumping up the leader
epoch, but failing to get enough votes. When the partitioning is removed,
this broker's high leader epoch will force a leader election. I assume
other Raft brokers can immediately advance their leader epoch passing the
already bumped epoch such that leader election won't be delayed. Is that
right?

205. In a JBOD setting, could we use the existing tool to move the Raft log
from one disk to another?

206. The KIP doesn't mention the local metadata store derived from the Raft
log. Will that be covered in a separate KIP?

207. Since this is a critical component. Could we add a section on the
testing plan for correctness?

208. Performance. Do we plan to do group commit (e.g. buffer pending
appends during a flush and then flush all accumulated pending records
together in the next flush) for better throughput?

209. "the leader can actually defer fsync until it knows "quorum.size - 1"
has get to a certain entry offset." Why is that "quorum.size - 1" instead
of the majority of the quorum?

Thanks,

Jun

On Mon, Jul 13, 2020 at 9:43 AM Jason Gustafson <ja...@confluent.io> wrote:

> Hi All,
>
> Just a quick update on the proposal. We have decided to move quorum
> reassignment to a separate KIP:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment
> .
> The way this ties into cluster bootstrapping is complicated, so we felt we
> needed a bit more time for validation. That leaves the core of this
> proposal as quorum-based replication. If there are no further comments, we
> will plan to start a vote later this week.
>
> Thanks,
> Jason
>
> On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com> wrote:
>
> > @Jun Rao <ju...@gmail.com>
> >
> > Regarding your comment about log compaction. After some deep-diving into
> > this we've decided to propose a new snapshot-based log cleaning mechanism
> > which would be used to replace the current compaction mechanism for this
> > meta log. A new KIP will be proposed specifically for this idea.
> >
> > All,
> >
> > I've updated the KIP wiki a bit updating one config "
> > election.jitter.max.ms"
> > to "election.backoff.max.ms" to make it more clear about the usage: the
> > configured value will be the upper bound of the binary exponential
> backoff
> > time after a failed election, before starting a new one.
> >
> >
> >
> > Guozhang
> >
> >
> >
> > On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> > > Thanks for the suggestions Guozhang.
> > >
> > > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hello Boyang,
> > > >
> > > > Thanks for the updated information. A few questions here:
> > > >
> > > > 1) Should the quorum-file also update to support multi-raft?
> > > >
> > > > I'm neutral about this, as we don't know yet how the multi-raft
> modules
> > > would behave. If
> > > we have different threads operating different raft groups,
> consolidating
> > > the `checkpoint` files seems
> > > not reasonable. We could always add `multi-quorum-file` later if
> > possible.
> > >
> > > 2) In the previous proposal, there's fields in the FetchQuorumRecords
> > like
> > > > latestDirtyOffset, is that dropped intentionally?
> > > >
> > > > I dropped the latestDirtyOffset since it is associated with the log
> > > compaction discussion. This is beyond this KIP scope and we could
> > > potentially get a separate KIP to talk about it.
> > >
> > >
> > > > 3) I think we also need to elaborate a bit more details regarding
> when
> > to
> > > > send metadata request and discover-brokers; currently we only
> discussed
> > > > during bootstrap how these requests would be sent. I think the
> > following
> > > > scenarios would also need these requests
> > > >
> > > > 3.a) As long as a broker does not know the current quorum (including
> > the
> > > > leader and the voters), it should continue periodically ask other
> > brokers
> > > > via "metadata.
> > > > 3.b) As long as a broker does not know all the current quorum voter's
> > > > connections, it should continue periodically ask other brokers via
> > > > "discover-brokers".
> > > > 3.c) When the leader's fetch timeout elapsed, it should send metadata
> > > > request.
> > > >
> > > > Make sense, will add to the KIP.
> > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> > reluctanthero104@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey all,
> > > > >
> > > > > follow-up on the previous email, we made some more updates:
> > > > >
> > > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > > > multi-raft.
> > > > >
> > > > > 2. We add observer status into the DescribeQuorumResponse as we see
> > it
> > > > is a
> > > > > low hanging fruit which is very useful for user debugging and
> > > > reassignment.
> > > > >
> > > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which
> is
> > > > purely
> > > > > in charge of discovering broker connections in a gossip manner. The
> > > > quorum
> > > > > leader discovery is piggy-back on the Metadata RPC for the topic
> > > > partition
> > > > > leader, which in our case is the single metadata partition for the
> > > > version
> > > > > one.
> > > > >
> > > > > Let me know if you have any questions.
> > > > >
> > > > > Boyang
> > > > >
> > > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey all,
> > > > > >
> > > > > > Thanks for the great discussions so far. I'm posting some KIP
> > updates
> > > > > from
> > > > > > our working group discussion:
> > > > > >
> > > > > > 1. We will be changing the core RPCs from single-raft API to
> > > > multi-raft.
> > > > > > This means all protocols will be "batch" in the first version,
> but
> > > the
> > > > > KIP
> > > > > > itself only illustrates the design for a single metadata topic
> > > > partition.
> > > > > > The reason is to "keep the door open" for future extensions of
> this
> > > > piece
> > > > > > of module such as a sharded controller or general quorum based
> > topic
> > > > > > replication, beyond the current Kafka replication protocol.
> > > > > >
> > > > > > 2. We will piggy-back on the current Kafka Fetch API instead of
> > > > inventing
> > > > > > a new FetchQuorumRecords RPC. The motivation is about the same as
> > #1
> > > as
> > > > > > well as making the integration work easier, instead of letting
> two
> > > > > similar
> > > > > > RPCs diverge.
> > > > > >
> > > > > > 3. In the EndQuorumEpoch protocol, instead of only sending the
> > > request
> > > > to
> > > > > > the most caught-up voter, we shall broadcast the information to
> all
> > > > > voters,
> > > > > > with a sorted voter list in descending order of their
> corresponding
> > > > > > replicated offset. In this way, the top voter will become a
> > candidate
> > > > > > immediately, while the other voters shall wait for an exponential
> > > > > back-off
> > > > > > to trigger elections, which helps ensure the top voter gets
> > elected,
> > > > and
> > > > > > the election eventually happens when the top voter is not
> > responsive.
> > > > > >
> > > > > > Please see the updated KIP and post any questions or concerns on
> > the
> > > > > > mailing thread.
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > >> Hi, Guozhang and Jason,
> > > > > >>
> > > > > >> Thanks for the reply. A couple of more replies.
> > > > > >>
> > > > > >> 102. Still not sure about this. How is the tombstone issue
> > addressed
> > > > in
> > > > > >> the
> > > > > >> non-voter and the observer.  They can die at any point and
> restart
> > > at
> > > > an
> > > > > >> arbitrary later time, and the advancing of the firstDirty offset
> > and
> > > > the
> > > > > >> removal of the tombstone can happen independently.
> > > > > >>
> > > > > >> 106. I agree that it would be less confusing if we used "epoch"
> > > > instead
> > > > > of
> > > > > >> "leader epoch" consistently.
> > > > > >>
> > > > > >> Jun
> > > > > >>
> > > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <
> wangguoz@gmail.com>
> > > > > wrote:
> > > > > >>
> > > > > >> > Thanks Jun. Further replies are in-lined.
> > > > > >> >
> > > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > >> >
> > > > > >> > > Hi, Guozhang,
> > > > > >> > >
> > > > > >> > > Thanks for the reply. A few more replies inlined below.
> > > > > >> > >
> > > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > >> wrote:
> > > > > >> > >
> > > > > >> > > > Hello Jun,
> > > > > >> > > >
> > > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > > >> > > >
> > > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > 101. Bootstrapping related issues.
> > > > > >> > > > > 101.1 Currently, we support auto broker id generation.
> Is
> > > this
> > > > > >> > > supported
> > > > > >> > > > > for bootstrap brokers?
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > > The vote ids would just be the broker ids.
> > "bootstrap.servers"
> > > > > >> would be
> > > > > >> > > > similar to what client configs have today, where
> > > "quorum.voters"
> > > > > >> would
> > > > > >> > be
> > > > > >> > > > pre-defined config values.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > My question was on the auto generated broker id. Currently,
> > the
> > > > > broker
> > > > > >> > can
> > > > > >> > > choose to have its broker Id auto generated. The generation
> is
> > > > done
> > > > > >> > through
> > > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how
> the
> > > > > broker
> > > > > >> id
> > > > > >> > is
> > > > > >> > > auto generated. "quorum.voters" also can't be set statically
> > if
> > > > > broker
> > > > > >> > ids
> > > > > >> > > are auto generated.
> > > > > >> > >
> > > > > >> > > Jason has explained some ideas that we've discussed so far,
> > the
> > > > > >> reason we
> > > > > >> > intentional did not include them so far is that we feel it is
> > > > out-side
> > > > > >> the
> > > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> > > definitely
> > > > > >> > address them though.
> > > > > >> >
> > > > > >> > On the high-level, our belief is that "joining a quorum" and
> > > > "joining
> > > > > >> (or
> > > > > >> > more specifically, registering brokers in) the cluster" would
> be
> > > > > >> > de-coupled a bit, where the former should be completed before
> we
> > > do
> > > > > the
> > > > > >> > latter. More specifically, assuming the quorum is already up
> and
> > > > > >> running,
> > > > > >> > after the newly started broker found the leader of the quorum
> it
> > > can
> > > > > >> send a
> > > > > >> > specific RegisterBroker request including its listener /
> > protocol
> > > /
> > > > > etc,
> > > > > >> > and upon handling it the leader can send back the uniquely
> > > generated
> > > > > >> broker
> > > > > >> > id to the new broker, while also executing the
> "startNewBroker"
> > > > > >> callback as
> > > > > >> > the controller.
> > > > > >> >
> > > > > >> >
> > > > > >> > >
> > > > > >> > > > > 102. Log compaction. One weak spot of log compaction is
> > for
> > > > the
> > > > > >> > > consumer
> > > > > >> > > > to
> > > > > >> > > > > deal with deletes. When a key is deleted, it's retained
> > as a
> > > > > >> > tombstone
> > > > > >> > > > > first and then physically removed. If a client misses
> the
> > > > > >> tombstone
> > > > > >> > > > > (because it's physically removed), it may not be able to
> > > > update
> > > > > >> its
> > > > > >> > > > > metadata properly. The way we solve this in Kafka is
> based
> > > on
> > > > a
> > > > > >> > > > > configuration (log.cleaner.delete.retention.ms) and we
> > > > expect a
> > > > > >> > > consumer
> > > > > >> > > > > having seen an old key to finish reading the deletion
> > > > tombstone
> > > > > >> > within
> > > > > >> > > > that
> > > > > >> > > > > time. There is no strong guarantee for that since a
> broker
> > > > could
> > > > > >> be
> > > > > >> > > down
> > > > > >> > > > > for a long time. It would be better if we can have a
> more
> > > > > reliable
> > > > > >> > way
> > > > > >> > > of
> > > > > >> > > > > dealing with deletes.
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > > We propose to capture this in the "FirstDirtyOffset" field
> > of
> > > > the
> > > > > >> > quorum
> > > > > >> > > > record fetch response: the offset is the maximum offset
> that
> > > log
> > > > > >> > > compaction
> > > > > >> > > > has reached up to. If the follower has fetched beyond this
> > > > offset
> > > > > it
> > > > > >> > > means
> > > > > >> > > > itself is safe hence it has seen all records up to that
> > > offset.
> > > > On
> > > > > >> > > getting
> > > > > >> > > > the response, the follower can then decide if its end
> offset
> > > > > >> actually
> > > > > >> > > below
> > > > > >> > > > that dirty offset (and hence may miss some tombstones). If
> > > > that's
> > > > > >> the
> > > > > >> > > case:
> > > > > >> > > >
> > > > > >> > > > 1) Naively, it could re-bootstrap metadata log from the
> very
> > > > > >> beginning
> > > > > >> > to
> > > > > >> > > > catch up.
> > > > > >> > > > 2) During that time, it would refrain itself from
> answering
> > > > > >> > > MetadataRequest
> > > > > >> > > > from any clients.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> > addresses
> > > > the
> > > > > >> > issue.
> > > > > >> > > Currently, the deletion tombstone is not removed immediately
> > > > after a
> > > > > >> > round
> > > > > >> > > of cleaning. It's removed after a delay in a subsequent
> round
> > of
> > > > > >> > cleaning.
> > > > > >> > > Consider an example where a key insertion is at offset 200
> > and a
> > > > > >> deletion
> > > > > >> > > tombstone of the key is at 400. Initially, FirstDirtyOffset
> is
> > > at
> > > > > >> 300. A
> > > > > >> > > follower/observer fetches from offset 0  and fetches the key
> > at
> > > > > offset
> > > > > >> > 200.
> > > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500
> > and
> > > > the
> > > > > >> > > tombstone at 400 is physically removed. The
> follower/observer
> > > > > >> continues
> > > > > >> > the
> > > > > >> > > fetch, but misses offset 400. It catches all the way to
> > > > > >> FirstDirtyOffset
> > > > > >> > > and declares its metadata as ready. However, its metadata
> > could
> > > be
> > > > > >> stale
> > > > > >> > > since it actually misses the deletion of the key.
> > > > > >> > >
> > > > > >> > > Yeah good question, I should have put more details in my
> > > > explanation
> > > > > >> :)
> > > > > >> >
> > > > > >> > The idea is that we will adjust the log compaction for this
> raft
> > > > based
> > > > > >> > metadata log: before more details to be explained, since we
> have
> > > two
> > > > > >> types
> > > > > >> > of "watermarks" here, whereas in Kafka the watermark indicates
> > > where
> > > > > >> every
> > > > > >> > replica have replicated up to and in Raft the watermark
> > indicates
> > > > > where
> > > > > >> the
> > > > > >> > majority of replicas (here only indicating voters of the
> quorum,
> > > not
> > > > > >> > counting observers) have replicated up to, let's call them
> Kafka
> > > > > >> watermark
> > > > > >> > and Raft watermark. For this special log, we would maintain
> both
> > > > > >> > watermarks.
> > > > > >> >
> > > > > >> > When log compacting on the leader, we would only compact up to
> > the
> > > > > Kafka
> > > > > >> > watermark, i.e. if there is at least one voter who have not
> > > > replicated
> > > > > >> an
> > > > > >> > entry, it would not be compacted. The "dirty-offset" is the
> > offset
> > > > > that
> > > > > >> > we've compacted up to and is communicated to other voters, and
> > the
> > > > > other
> > > > > >> > voters would also compact up to this value --- i.e. the
> > difference
> > > > > here
> > > > > >> is
> > > > > >> > that instead of letting each replica doing log compaction
> > > > > independently,
> > > > > >> > we'll have the leader to decide upon which offset to compact
> to,
> > > and
> > > > > >> > propagate this value to others to follow, in a more
> coordinated
> > > > > manner.
> > > > > >> > Also note when there are new voters joining the quorum who has
> > not
> > > > > >> > replicated up to the dirty-offset, of because of other issues
> > they
> > > > > >> > truncated their logs to below the dirty-offset, they'd have to
> > > > > >> re-bootstrap
> > > > > >> > from the beginning, and during this period of time the leader
> > > > learned
> > > > > >> about
> > > > > >> > this lagging voter would not advance the watermark (also it
> > would
> > > > not
> > > > > >> > decrement it), and hence not compacting either, until the
> > voter(s)
> > > > has
> > > > > >> > caught up to that dirty-offset.
> > > > > >> >
> > > > > >> > So back to your example above, before the bootstrap voter gets
> > to
> > > > 300
> > > > > no
> > > > > >> > log compaction would happen on the leader; and until later
> when
> > > the
> > > > > >> voter
> > > > > >> > have got to beyond 400 and hence replicated that tombstone,
> the
> > > log
> > > > > >> > compaction would possibly get to that tombstone and remove it.
> > Say
> > > > > >> later it
> > > > > >> > the leader's log compaction reaches 500, it can send this back
> > to
> > > > the
> > > > > >> voter
> > > > > >> > who can then also compact locally up to 500.
> > > > > >> >
> > > > > >> >
> > > > > >> > > > > 105. Quorum State: In addition to VotedId, do we need
> the
> > > > epoch
> > > > > >> > > > > corresponding to VotedId? Over time, the same broker Id
> > > could
> > > > be
> > > > > >> > voted
> > > > > >> > > in
> > > > > >> > > > > different generations with different epoch.
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > Hmm, this is a good point. Originally I think the
> > > "LeaderEpoch"
> > > > > >> field
> > > > > >> > in
> > > > > >> > > > that file is corresponding to the "latest known leader
> > epoch",
> > > > not
> > > > > >> the
> > > > > >> > > > "current leader epoch". For example, if the current epoch
> is
> > > N,
> > > > > and
> > > > > >> > then
> > > > > >> > > a
> > > > > >> > > > vote-request with epoch N+1 is received and the voter
> > granted
> > > > the
> > > > > >> vote
> > > > > >> > > for
> > > > > >> > > > it, then it means for this voter it knows the "latest
> epoch"
> > > is
> > > > N
> > > > > +
> > > > > >> 1
> > > > > >> > > > although it is unknown if that sending candidate will
> indeed
> > > > > become
> > > > > >> the
> > > > > >> > > new
> > > > > >> > > > leader (which would only be notified via begin-quorum
> > > request).
> > > > > >> > However,
> > > > > >> > > > when persisting the quorum state, we would encode
> > leader-epoch
> > > > to
> > > > > >> N+1,
> > > > > >> > > > while the leaderId to be the older leader.
> > > > > >> > > >
> > > > > >> > > > But now thinking about this a bit more, I feel we should
> use
> > > two
> > > > > >> > separate
> > > > > >> > > > epochs, one for the "lates known" and one for the
> "current"
> > to
> > > > > pair
> > > > > >> > with
> > > > > >> > > > the leaderId. I will update the wiki page.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > Hmm, it's kind of weird to bump up the leader epoch before
> the
> > > new
> > > > > >> leader
> > > > > >> > > is actually elected, right.
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the
> FetchQuorumRecords
> > > API
> > > > to
> > > > > >> > > indicate
> > > > > >> > > > > that the follower has fetched from an invalid offset and
> > > > should
> > > > > >> > > truncate
> > > > > >> > > > to
> > > > > >> > > > > the offset/epoch indicated in the response." Observers
> > can't
> > > > > >> truncate
> > > > > >> > > > their
> > > > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > I'm not sure if I understand your question? Observers
> should
> > > > still
> > > > > >> be
> > > > > >> > > able
> > > > > >> > > > to truncate their logs as well.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > Hmm, I thought only the quorum nodes have local logs and
> > > observers
> > > > > >> don't?
> > > > > >> > >
> > > > > >> > > > 107. "The leader will continue sending BeginQuorumEpoch to
> > > each
> > > > > >> known
> > > > > >> > > > voter
> > > > > >> > > > > until it has received its endorsement." If a voter is
> down
> > > > for a
> > > > > >> long
> > > > > >> > > > time,
> > > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> > overhead.
> > > > > >> > Similarly,
> > > > > >> > > > if a
> > > > > >> > > > > follower stops sending FetchQuorumRecords, does the
> leader
> > > > keep
> > > > > >> > sending
> > > > > >> > > > > BeginQuorumEpoch?
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > > > >> > begin-quorum-epoch
> > > > > >> > > > request is for voters to quickly get the new leader
> > > information;
> > > > > >> > however
> > > > > >> > > > even if they do not get them they can still eventually
> learn
> > > > about
> > > > > >> that
> > > > > >> > > > from others via gossiping FindQuorum. I think we can
> adjust
> > > the
> > > > > >> logic
> > > > > >> > to
> > > > > >> > > > e.g. exponential back-off or with a limited num.retries.
> > > > > >> > > >
> > > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > > >> FetchQuorumRecords
> > > > > >> > > > already, it means that follower already knows that the
> > broker
> > > is
> > > > > the
> > > > > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch;
> > > however
> > > > > it
> > > > > >> is
> > > > > >> > > > possible that after a follower sends FetchQuorumRecords
> > > already,
> > > > > >> > suddenly
> > > > > >> > > > it stops send it (possibly because it learned about a
> higher
> > > > epoch
> > > > > >> > > leader),
> > > > > >> > > > and hence this broker may be a "zombie" leader and we
> > propose
> > > to
> > > > > use
> > > > > >> > the
> > > > > >> > > > fetch.timeout to let the leader to try to verify if it has
> > > > already
> > > > > >> been
> > > > > >> > > > stale.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > It just seems that we should handle these two cases in a
> > > > consistent
> > > > > >> way?
> > > > > >> > >
> > > > > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords
> > from a
> > > > > >> follower
> > > > > >> > could mean that we no longer needs to send BeginQuorumEpoch
> > > anymore
> > > > > ---
> > > > > >> and
> > > > > >> > it is already part of our current implementations in
> > > > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > > >> >
> > > > > >> >
> > > > > >> > > Thanks,
> > > > > >> > >
> > > > > >> > > Jun
> > > > > >> > >
> > > > > >> > > >
> > > > > >> > > > >
> > > > > >> > > > > Jun
> > > > > >> > > > >
> > > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > >> >
> > > > > >> > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > Hello Leonard,
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks for your comments, I'm relying in line below:
> > > > > >> > > > > >
> > > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > > > > >> > wge@confluent.io>
> > > > > >> > > > > > wrote:
> > > > > >> > > > > >
> > > > > >> > > > > > > Hi Kafka developers,
> > > > > >> > > > > > >
> > > > > >> > > > > > > It's great to see this proposal and it took me some
> > time
> > > > to
> > > > > >> > finish
> > > > > >> > > > > > reading
> > > > > >> > > > > > > it.
> > > > > >> > > > > > >
> > > > > >> > > > > > > And I have the following questions about the
> Proposal:
> > > > > >> > > > > > >
> > > > > >> > > > > > >    - How do we plan to test this design to ensure
> its
> > > > > >> > correctness?
> > > > > >> > > Or
> > > > > >> > > > > > more
> > > > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’
> based
> > > > model
> > > > > >> is
> > > > > >> > > > > > functional
> > > > > >> > > > > > > and
> > > > > >> > > > > > >    correct given that it is different from the
> > original
> > > > RAFT
> > > > > >> > > > > > implementation
> > > > > >> > > > > > >    which has formal proof of correctness?
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > We have two planned verifications on the correctness
> and
> > > > > >> liveness
> > > > > >> > of
> > > > > >> > > > the
> > > > > >> > > > > > design. One is via model verification (TLA+)
> > > > > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > > > > >> > > > > >
> > > > > >> > > > > > Another is via the concurrent simulation tests
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > >> > > > > >
> > > > > >> > > > > >    - Have we considered any sensible defaults for the
> > > > > >> > configuration,
> > > > > >> > > > i.e.
> > > > > >> > > > > > >    all the election timeout, fetch time out, etc.?
> Or
> > we
> > > > > want
> > > > > >> to
> > > > > >> > > > leave
> > > > > >> > > > > > > this to
> > > > > >> > > > > > >    a later stage when we do the performance testing,
> > > etc.
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > This is a good question, the reason we did not set any
> > > > default
> > > > > >> > values
> > > > > >> > > > for
> > > > > >> > > > > > the timeout configurations is that we think it may
> take
> > > some
> > > > > >> > > > benchmarking
> > > > > >> > > > > > experiments to get these defaults right. Some
> high-level
> > > > > >> principles
> > > > > >> > > to
> > > > > >> > > > > > consider: 1) the fetch.timeout should be around the
> same
> > > > scale
> > > > > >> with
> > > > > >> > > zk
> > > > > >> > > > > > session timeout, which is now 18 seconds by default --
> > in
> > > > > >> practice
> > > > > >> > > > we've
> > > > > >> > > > > > seen unstable networks having more than 10 secs of
> > > transient
> > > > > >> > > > > connectivity,
> > > > > >> > > > > > 2) the election.timeout, however, should be smaller
> than
> > > the
> > > > > >> fetch
> > > > > >> > > > > timeout
> > > > > >> > > > > > as is also suggested as a practical optimization in
> > > > > literature:
> > > > > >> > > > > >
> > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > >> > > > > >
> > > > > >> > > > > > Some more discussions can be found here:
> > > > > >> > > > > >
> > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > >    - Have we considered piggybacking
> > `BeginQuorumEpoch`
> > > > with
> > > > > >> the
> > > > > >> > `
> > > > > >> > > > > > >    FetchQuorumRecords`? I might be missing something
> > > > obvious
> > > > > >> but
> > > > > >> > I
> > > > > >> > > am
> > > > > >> > > > > > just
> > > > > >> > > > > > >    wondering why don’t we just use the `FindQuorum`
> > and
> > > > > >> > > > > > > `FetchQuorumRecords`
> > > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader ->
> > > other
> > > > > >> voter
> > > > > >> > > > > > followers, while FindQuorum / Fetch are sent from
> > follower
> > > > to
> > > > > >> > leader.
> > > > > >> > > > > > Arguably one can eventually realize the new leader and
> > > epoch
> > > > > via
> > > > > >> > > > > gossiping
> > > > > >> > > > > > FindQuorum, but that could in practice require a long
> > > delay.
> > > > > >> > Having a
> > > > > >> > > > > > leader -> other voters request helps the new leader
> > epoch
> > > to
> > > > > be
> > > > > >> > > > > propagated
> > > > > >> > > > > > faster under a pull model.
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > >    - And about the `FetchQuorumRecords` response
> > schema,
> > > > in
> > > > > >> the
> > > > > >> > > > > `Records`
> > > > > >> > > > > > >    field of the response, is it just one record or
> all
> > > the
> > > > > >> > records
> > > > > >> > > > > > starting
> > > > > >> > > > > > >    from the FetchOffset? It seems a lot more
> efficient
> > > if
> > > > we
> > > > > >> sent
> > > > > >> > > all
> > > > > >> > > > > the
> > > > > >> > > > > > >    records during the bootstrapping of the brokers.
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> > > > starting
> > > > > >> > offset
> > > > > >> > > of
> > > > > >> > > > > the
> > > > > >> > > > > > batch of records.
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > >    - Regarding the disruptive broker issues, does
> our
> > > pull
> > > > > >> based
> > > > > >> > > > model
> > > > > >> > > > > > >    suffer from it? If so, have we considered the
> > > Pre-Vote
> > > > > >> stage?
> > > > > >> > If
> > > > > >> > > > > not,
> > > > > >> > > > > > > why?
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > The disruptive broker is stated in the original Raft
> > paper
> > > > > >> which is
> > > > > >> > > the
> > > > > >> > > > > > result of the push model design. Our analysis showed
> > that
> > > > with
> > > > > >> the
> > > > > >> > > pull
> > > > > >> > > > > > model it is no longer an issue.
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > > Thanks a lot for putting this up, and I hope that my
> > > > > questions
> > > > > >> > can
> > > > > >> > > be
> > > > > >> > > > > of
> > > > > >> > > > > > > some value to make this KIP better.
> > > > > >> > > > > > >
> > > > > >> > > > > > > Hope to hear from you soon!
> > > > > >> > > > > > >
> > > > > >> > > > > > > Best wishes,
> > > > > >> > > > > > > Leonard
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > > >> cmccabe@apache.org
> > > > > >> > >
> > > > > >> > > > > wrote:
> > > > > >> > > > > > >
> > > > > >> > > > > > > > Hi Jason,
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > I haven't had a chance to read in detail, but I
> read
> > > the
> > > > > >> > outline
> > > > > >> > > > and
> > > > > >> > > > > a
> > > > > >> > > > > > > few
> > > > > >> > > > > > > > things jumped out at me.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > First, for every epoch that is 32 bits rather than
> > > 64, I
> > > > > >> sort
> > > > > >> > of
> > > > > >> > > > > wonder
> > > > > >> > > > > > > if
> > > > > >> > > > > > > > that's a good long-term choice.  I keep reading
> > about
> > > > > stuff
> > > > > >> > like
> > > > > >> > > > > this:
> > > > > >> > > > > > > >
> > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > > .
> > > > > >> > > Obviously,
> > > > > >> > > > > > that
> > > > > >> > > > > > > > JIRA is about zxid, which increments much faster
> > than
> > > we
> > > > > >> expect
> > > > > >> > > > these
> > > > > >> > > > > > > > leader epochs to, but it would still be good to
> see
> > > some
> > > > > >> rough
> > > > > >> > > > > > > calculations
> > > > > >> > > > > > > > about how long 32 bits (or really, 31 bits) will
> > last
> > > us
> > > > > in
> > > > > >> the
> > > > > >> > > > cases
> > > > > >> > > > > > > where
> > > > > >> > > > > > > > we're using it, and what the space savings we're
> > > getting
> > > > > >> really
> > > > > >> > > is.
> > > > > >> > > > > It
> > > > > >> > > > > > > > seems like in most cases the tradeoff may not be
> > worth
> > > > it?
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Another thing I've been thinking about is how we
> do
> > > > > >> > > > bootstrapping.  I
> > > > > >> > > > > > > > would prefer to be in a world where formatting a
> new
> > > > Kafka
> > > > > >> node
> > > > > >> > > > was a
> > > > > >> > > > > > > first
> > > > > >> > > > > > > > class operation explicitly initiated by the admin,
> > > > rather
> > > > > >> than
> > > > > >> > > > > > something
> > > > > >> > > > > > > > that happened implicitly when you started up the
> > > broker
> > > > > and
> > > > > >> > > things
> > > > > >> > > > > > > "looked
> > > > > >> > > > > > > > blank."
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > The first problem is that things can "look blank"
> > > > > >> accidentally
> > > > > >> > if
> > > > > >> > > > the
> > > > > >> > > > > > > > storage system is having a bad day.  Clearly in
> the
> > > > > non-Raft
> > > > > >> > > world,
> > > > > >> > > > > > this
> > > > > >> > > > > > > > leads to data loss if the broker that is
> (re)started
> > > > this
> > > > > >> way
> > > > > >> > was
> > > > > >> > > > the
> > > > > >> > > > > > > > leader for some partitions.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > The second problem is that we have a bit of a
> > chicken
> > > > and
> > > > > >> egg
> > > > > >> > > > problem
> > > > > >> > > > > > > with
> > > > > >> > > > > > > > certain configuration keys.  For example, maybe
> you
> > > want
> > > > > to
> > > > > >> > > > configure
> > > > > >> > > > > > > some
> > > > > >> > > > > > > > connection security settings in your cluster, but
> > you
> > > > > don't
> > > > > >> > want
> > > > > >> > > > them
> > > > > >> > > > > > to
> > > > > >> > > > > > > > ever be stored in a plaintext config file.  (For
> > > > example,
> > > > > >> SCRAM
> > > > > >> > > > > > > passwords,
> > > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > > configuration,
> > > > > >> but
> > > > > >> > > > that
> > > > > >> > > > > > > brings
> > > > > >> > > > > > > > up the chicken and egg problem.  The broker needs
> to
> > > be
> > > > > >> > > configured
> > > > > >> > > > to
> > > > > >> > > > > > > know
> > > > > >> > > > > > > > how to talk to you, but you need to configure it
> > > before
> > > > > you
> > > > > >> can
> > > > > >> > > > talk
> > > > > >> > > > > to
> > > > > >> > > > > > > > it.  Using an external secret manager like Vault
> is
> > > one
> > > > > way
> > > > > >> to
> > > > > >> > > > solve
> > > > > >> > > > > > > this,
> > > > > >> > > > > > > > but not everyone uses an external secret manager.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > quorum.voters seems like a similar configuration
> > key.
> > > > In
> > > > > >> the
> > > > > >> > > > current
> > > > > >> > > > > > > KIP,
> > > > > >> > > > > > > > this is only read if there is no other
> configuration
> > > > > >> specifying
> > > > > >> > > the
> > > > > >> > > > > > > quorum
> > > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> > > wouldn't
> > > > > need
> > > > > >> > this
> > > > > >> > > > key
> > > > > >> > > > > > > > because we could assume that there was always
> quorum
> > > > > >> > information
> > > > > >> > > > > stored
> > > > > >> > > > > > > > locally.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > best,
> > > > > >> > > > > > > > Colin
> > > > > >> > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson
> > wrote:
> > > > > >> > > > > > > > > Hi All,
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > >> > > > > > > > .
> > > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > > ultimately
> > > > > >> replace
> > > > > >> > > > > > Zookeeper
> > > > > >> > > > > > > > > as
> > > > > >> > > > > > > > > documented in KIP-500. Please take a look and
> > share
> > > > your
> > > > > >> > > > thoughts.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > A few minor notes to set the stage a little bit:
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > - This KIP does not specify the structure of the
> > > > > messages
> > > > > >> > used
> > > > > >> > > to
> > > > > >> > > > > > > > represent
> > > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> > internal
> > > > API
> > > > > >> that
> > > > > >> > > will
> > > > > >> > > > > be
> > > > > >> > > > > > > used
> > > > > >> > > > > > > > > by the controller. Expect these to come in later
> > > > > >> proposals.
> > > > > >> > > Here
> > > > > >> > > > we
> > > > > >> > > > > > are
> > > > > >> > > > > > > > > primarily concerned with the replication
> protocol
> > > and
> > > > > >> basic
> > > > > >> > > > > > operational
> > > > > >> > > > > > > > > mechanics.
> > > > > >> > > > > > > > > - We expect many details to change as we get
> > closer
> > > to
> > > > > >> > > > integration
> > > > > >> > > > > > with
> > > > > >> > > > > > > > > the controller. Any changes we make will be made
> > > > either
> > > > > as
> > > > > >> > > > > amendments
> > > > > >> > > > > > > to
> > > > > >> > > > > > > > > this KIP or, in the case of larger changes, as
> new
> > > > > >> proposals.
> > > > > >> > > > > > > > > - We have a prototype implementation which I
> will
> > > put
> > > > > >> online
> > > > > >> > > > within
> > > > > >> > > > > > the
> > > > > >> > > > > > > > > next week which may help in understanding some
> > > > details.
> > > > > It
> > > > > >> > has
> > > > > >> > > > > > > diverged a
> > > > > >> > > > > > > > > little bit from our proposal, so I am taking a
> > > little
> > > > > >> time to
> > > > > >> > > > bring
> > > > > >> > > > > > it
> > > > > >> > > > > > > in
> > > > > >> > > > > > > > > line. I'll post an update to this thread when it
> > is
> > > > > >> available
> > > > > >> > > for
> > > > > >> > > > > > > review.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > Finally, I want to mention that this proposal
> was
> > > > > drafted
> > > > > >> by
> > > > > >> > > > > myself,
> > > > > >> > > > > > > > Boyang
> > > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > Thanks,
> > > > > >> > > > > > > > > Jason
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > --
> > > > > >> > > > > > > Leonard Ge
> > > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > --
> > > > > >> > > > > > -- Guozhang
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > --
> > > > > >> > > > -- Guozhang
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >> >
> > > > > >> > --
> > > > > >> > -- Guozhang
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jason Gustafson <ja...@confluent.io>.
Hi All,

Just a quick update on the proposal. We have decided to move quorum
reassignment to a separate KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-642%3A+Dynamic+quorum+reassignment.
The way this ties into cluster bootstrapping is complicated, so we felt we
needed a bit more time for validation. That leaves the core of this
proposal as quorum-based replication. If there are no further comments, we
will plan to start a vote later this week.

Thanks,
Jason

On Wed, Jun 24, 2020 at 10:43 AM Guozhang Wang <wa...@gmail.com> wrote:

> @Jun Rao <ju...@gmail.com>
>
> Regarding your comment about log compaction. After some deep-diving into
> this we've decided to propose a new snapshot-based log cleaning mechanism
> which would be used to replace the current compaction mechanism for this
> meta log. A new KIP will be proposed specifically for this idea.
>
> All,
>
> I've updated the KIP wiki a bit updating one config "
> election.jitter.max.ms"
> to "election.backoff.max.ms" to make it more clear about the usage: the
> configured value will be the upper bound of the binary exponential backoff
> time after a failed election, before starting a new one.
>
>
>
> Guozhang
>
>
>
> On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <re...@gmail.com>
> wrote:
>
> > Thanks for the suggestions Guozhang.
> >
> > On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Hello Boyang,
> > >
> > > Thanks for the updated information. A few questions here:
> > >
> > > 1) Should the quorum-file also update to support multi-raft?
> > >
> > > I'm neutral about this, as we don't know yet how the multi-raft modules
> > would behave. If
> > we have different threads operating different raft groups, consolidating
> > the `checkpoint` files seems
> > not reasonable. We could always add `multi-quorum-file` later if
> possible.
> >
> > 2) In the previous proposal, there's fields in the FetchQuorumRecords
> like
> > > latestDirtyOffset, is that dropped intentionally?
> > >
> > > I dropped the latestDirtyOffset since it is associated with the log
> > compaction discussion. This is beyond this KIP scope and we could
> > potentially get a separate KIP to talk about it.
> >
> >
> > > 3) I think we also need to elaborate a bit more details regarding when
> to
> > > send metadata request and discover-brokers; currently we only discussed
> > > during bootstrap how these requests would be sent. I think the
> following
> > > scenarios would also need these requests
> > >
> > > 3.a) As long as a broker does not know the current quorum (including
> the
> > > leader and the voters), it should continue periodically ask other
> brokers
> > > via "metadata.
> > > 3.b) As long as a broker does not know all the current quorum voter's
> > > connections, it should continue periodically ask other brokers via
> > > "discover-brokers".
> > > 3.c) When the leader's fetch timeout elapsed, it should send metadata
> > > request.
> > >
> > > Make sense, will add to the KIP.
> >
> > >
> > > Guozhang
> > >
> > >
> > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <
> reluctanthero104@gmail.com>
> > > wrote:
> > >
> > > > Hey all,
> > > >
> > > > follow-up on the previous email, we made some more updates:
> > > >
> > > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > > multi-raft.
> > > >
> > > > 2. We add observer status into the DescribeQuorumResponse as we see
> it
> > > is a
> > > > low hanging fruit which is very useful for user debugging and
> > > reassignment.
> > > >
> > > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which is
> > > purely
> > > > in charge of discovering broker connections in a gossip manner. The
> > > quorum
> > > > leader discovery is piggy-back on the Metadata RPC for the topic
> > > partition
> > > > leader, which in our case is the single metadata partition for the
> > > version
> > > > one.
> > > >
> > > > Let me know if you have any questions.
> > > >
> > > > Boyang
> > > >
> > > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> > reluctanthero104@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey all,
> > > > >
> > > > > Thanks for the great discussions so far. I'm posting some KIP
> updates
> > > > from
> > > > > our working group discussion:
> > > > >
> > > > > 1. We will be changing the core RPCs from single-raft API to
> > > multi-raft.
> > > > > This means all protocols will be "batch" in the first version, but
> > the
> > > > KIP
> > > > > itself only illustrates the design for a single metadata topic
> > > partition.
> > > > > The reason is to "keep the door open" for future extensions of this
> > > piece
> > > > > of module such as a sharded controller or general quorum based
> topic
> > > > > replication, beyond the current Kafka replication protocol.
> > > > >
> > > > > 2. We will piggy-back on the current Kafka Fetch API instead of
> > > inventing
> > > > > a new FetchQuorumRecords RPC. The motivation is about the same as
> #1
> > as
> > > > > well as making the integration work easier, instead of letting two
> > > > similar
> > > > > RPCs diverge.
> > > > >
> > > > > 3. In the EndQuorumEpoch protocol, instead of only sending the
> > request
> > > to
> > > > > the most caught-up voter, we shall broadcast the information to all
> > > > voters,
> > > > > with a sorted voter list in descending order of their corresponding
> > > > > replicated offset. In this way, the top voter will become a
> candidate
> > > > > immediately, while the other voters shall wait for an exponential
> > > > back-off
> > > > > to trigger elections, which helps ensure the top voter gets
> elected,
> > > and
> > > > > the election eventually happens when the top voter is not
> responsive.
> > > > >
> > > > > Please see the updated KIP and post any questions or concerns on
> the
> > > > > mailing thread.
> > > > >
> > > > > Boyang
> > > > >
> > > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > >> Hi, Guozhang and Jason,
> > > > >>
> > > > >> Thanks for the reply. A couple of more replies.
> > > > >>
> > > > >> 102. Still not sure about this. How is the tombstone issue
> addressed
> > > in
> > > > >> the
> > > > >> non-voter and the observer.  They can die at any point and restart
> > at
> > > an
> > > > >> arbitrary later time, and the advancing of the firstDirty offset
> and
> > > the
> > > > >> removal of the tombstone can happen independently.
> > > > >>
> > > > >> 106. I agree that it would be less confusing if we used "epoch"
> > > instead
> > > > of
> > > > >> "leader epoch" consistently.
> > > > >>
> > > > >> Jun
> > > > >>
> > > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com>
> > > > wrote:
> > > > >>
> > > > >> > Thanks Jun. Further replies are in-lined.
> > > > >> >
> > > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io>
> wrote:
> > > > >> >
> > > > >> > > Hi, Guozhang,
> > > > >> > >
> > > > >> > > Thanks for the reply. A few more replies inlined below.
> > > > >> > >
> > > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Hello Jun,
> > > > >> > > >
> > > > >> > > > Thanks for your comments! I'm replying inline below:
> > > > >> > > >
> > > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > >> > > >
> > > > >> > > > > 101. Bootstrapping related issues.
> > > > >> > > > > 101.1 Currently, we support auto broker id generation. Is
> > this
> > > > >> > > supported
> > > > >> > > > > for bootstrap brokers?
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > > The vote ids would just be the broker ids.
> "bootstrap.servers"
> > > > >> would be
> > > > >> > > > similar to what client configs have today, where
> > "quorum.voters"
> > > > >> would
> > > > >> > be
> > > > >> > > > pre-defined config values.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > My question was on the auto generated broker id. Currently,
> the
> > > > broker
> > > > >> > can
> > > > >> > > choose to have its broker Id auto generated. The generation is
> > > done
> > > > >> > through
> > > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how the
> > > > broker
> > > > >> id
> > > > >> > is
> > > > >> > > auto generated. "quorum.voters" also can't be set statically
> if
> > > > broker
> > > > >> > ids
> > > > >> > > are auto generated.
> > > > >> > >
> > > > >> > > Jason has explained some ideas that we've discussed so far,
> the
> > > > >> reason we
> > > > >> > intentional did not include them so far is that we feel it is
> > > out-side
> > > > >> the
> > > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> > definitely
> > > > >> > address them though.
> > > > >> >
> > > > >> > On the high-level, our belief is that "joining a quorum" and
> > > "joining
> > > > >> (or
> > > > >> > more specifically, registering brokers in) the cluster" would be
> > > > >> > de-coupled a bit, where the former should be completed before we
> > do
> > > > the
> > > > >> > latter. More specifically, assuming the quorum is already up and
> > > > >> running,
> > > > >> > after the newly started broker found the leader of the quorum it
> > can
> > > > >> send a
> > > > >> > specific RegisterBroker request including its listener /
> protocol
> > /
> > > > etc,
> > > > >> > and upon handling it the leader can send back the uniquely
> > generated
> > > > >> broker
> > > > >> > id to the new broker, while also executing the "startNewBroker"
> > > > >> callback as
> > > > >> > the controller.
> > > > >> >
> > > > >> >
> > > > >> > >
> > > > >> > > > > 102. Log compaction. One weak spot of log compaction is
> for
> > > the
> > > > >> > > consumer
> > > > >> > > > to
> > > > >> > > > > deal with deletes. When a key is deleted, it's retained
> as a
> > > > >> > tombstone
> > > > >> > > > > first and then physically removed. If a client misses the
> > > > >> tombstone
> > > > >> > > > > (because it's physically removed), it may not be able to
> > > update
> > > > >> its
> > > > >> > > > > metadata properly. The way we solve this in Kafka is based
> > on
> > > a
> > > > >> > > > > configuration (log.cleaner.delete.retention.ms) and we
> > > expect a
> > > > >> > > consumer
> > > > >> > > > > having seen an old key to finish reading the deletion
> > > tombstone
> > > > >> > within
> > > > >> > > > that
> > > > >> > > > > time. There is no strong guarantee for that since a broker
> > > could
> > > > >> be
> > > > >> > > down
> > > > >> > > > > for a long time. It would be better if we can have a more
> > > > reliable
> > > > >> > way
> > > > >> > > of
> > > > >> > > > > dealing with deletes.
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > > We propose to capture this in the "FirstDirtyOffset" field
> of
> > > the
> > > > >> > quorum
> > > > >> > > > record fetch response: the offset is the maximum offset that
> > log
> > > > >> > > compaction
> > > > >> > > > has reached up to. If the follower has fetched beyond this
> > > offset
> > > > it
> > > > >> > > means
> > > > >> > > > itself is safe hence it has seen all records up to that
> > offset.
> > > On
> > > > >> > > getting
> > > > >> > > > the response, the follower can then decide if its end offset
> > > > >> actually
> > > > >> > > below
> > > > >> > > > that dirty offset (and hence may miss some tombstones). If
> > > that's
> > > > >> the
> > > > >> > > case:
> > > > >> > > >
> > > > >> > > > 1) Naively, it could re-bootstrap metadata log from the very
> > > > >> beginning
> > > > >> > to
> > > > >> > > > catch up.
> > > > >> > > > 2) During that time, it would refrain itself from answering
> > > > >> > > MetadataRequest
> > > > >> > > > from any clients.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > I am not sure that the "FirstDirtyOffset" field fully
> addresses
> > > the
> > > > >> > issue.
> > > > >> > > Currently, the deletion tombstone is not removed immediately
> > > after a
> > > > >> > round
> > > > >> > > of cleaning. It's removed after a delay in a subsequent round
> of
> > > > >> > cleaning.
> > > > >> > > Consider an example where a key insertion is at offset 200
> and a
> > > > >> deletion
> > > > >> > > tombstone of the key is at 400. Initially, FirstDirtyOffset is
> > at
> > > > >> 300. A
> > > > >> > > follower/observer fetches from offset 0  and fetches the key
> at
> > > > offset
> > > > >> > 200.
> > > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500
> and
> > > the
> > > > >> > > tombstone at 400 is physically removed. The follower/observer
> > > > >> continues
> > > > >> > the
> > > > >> > > fetch, but misses offset 400. It catches all the way to
> > > > >> FirstDirtyOffset
> > > > >> > > and declares its metadata as ready. However, its metadata
> could
> > be
> > > > >> stale
> > > > >> > > since it actually misses the deletion of the key.
> > > > >> > >
> > > > >> > > Yeah good question, I should have put more details in my
> > > explanation
> > > > >> :)
> > > > >> >
> > > > >> > The idea is that we will adjust the log compaction for this raft
> > > based
> > > > >> > metadata log: before more details to be explained, since we have
> > two
> > > > >> types
> > > > >> > of "watermarks" here, whereas in Kafka the watermark indicates
> > where
> > > > >> every
> > > > >> > replica have replicated up to and in Raft the watermark
> indicates
> > > > where
> > > > >> the
> > > > >> > majority of replicas (here only indicating voters of the quorum,
> > not
> > > > >> > counting observers) have replicated up to, let's call them Kafka
> > > > >> watermark
> > > > >> > and Raft watermark. For this special log, we would maintain both
> > > > >> > watermarks.
> > > > >> >
> > > > >> > When log compacting on the leader, we would only compact up to
> the
> > > > Kafka
> > > > >> > watermark, i.e. if there is at least one voter who have not
> > > replicated
> > > > >> an
> > > > >> > entry, it would not be compacted. The "dirty-offset" is the
> offset
> > > > that
> > > > >> > we've compacted up to and is communicated to other voters, and
> the
> > > > other
> > > > >> > voters would also compact up to this value --- i.e. the
> difference
> > > > here
> > > > >> is
> > > > >> > that instead of letting each replica doing log compaction
> > > > independently,
> > > > >> > we'll have the leader to decide upon which offset to compact to,
> > and
> > > > >> > propagate this value to others to follow, in a more coordinated
> > > > manner.
> > > > >> > Also note when there are new voters joining the quorum who has
> not
> > > > >> > replicated up to the dirty-offset, of because of other issues
> they
> > > > >> > truncated their logs to below the dirty-offset, they'd have to
> > > > >> re-bootstrap
> > > > >> > from the beginning, and during this period of time the leader
> > > learned
> > > > >> about
> > > > >> > this lagging voter would not advance the watermark (also it
> would
> > > not
> > > > >> > decrement it), and hence not compacting either, until the
> voter(s)
> > > has
> > > > >> > caught up to that dirty-offset.
> > > > >> >
> > > > >> > So back to your example above, before the bootstrap voter gets
> to
> > > 300
> > > > no
> > > > >> > log compaction would happen on the leader; and until later when
> > the
> > > > >> voter
> > > > >> > have got to beyond 400 and hence replicated that tombstone, the
> > log
> > > > >> > compaction would possibly get to that tombstone and remove it.
> Say
> > > > >> later it
> > > > >> > the leader's log compaction reaches 500, it can send this back
> to
> > > the
> > > > >> voter
> > > > >> > who can then also compact locally up to 500.
> > > > >> >
> > > > >> >
> > > > >> > > > > 105. Quorum State: In addition to VotedId, do we need the
> > > epoch
> > > > >> > > > > corresponding to VotedId? Over time, the same broker Id
> > could
> > > be
> > > > >> > voted
> > > > >> > > in
> > > > >> > > > > different generations with different epoch.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > Hmm, this is a good point. Originally I think the
> > "LeaderEpoch"
> > > > >> field
> > > > >> > in
> > > > >> > > > that file is corresponding to the "latest known leader
> epoch",
> > > not
> > > > >> the
> > > > >> > > > "current leader epoch". For example, if the current epoch is
> > N,
> > > > and
> > > > >> > then
> > > > >> > > a
> > > > >> > > > vote-request with epoch N+1 is received and the voter
> granted
> > > the
> > > > >> vote
> > > > >> > > for
> > > > >> > > > it, then it means for this voter it knows the "latest epoch"
> > is
> > > N
> > > > +
> > > > >> 1
> > > > >> > > > although it is unknown if that sending candidate will indeed
> > > > become
> > > > >> the
> > > > >> > > new
> > > > >> > > > leader (which would only be notified via begin-quorum
> > request).
> > > > >> > However,
> > > > >> > > > when persisting the quorum state, we would encode
> leader-epoch
> > > to
> > > > >> N+1,
> > > > >> > > > while the leaderId to be the older leader.
> > > > >> > > >
> > > > >> > > > But now thinking about this a bit more, I feel we should use
> > two
> > > > >> > separate
> > > > >> > > > epochs, one for the "lates known" and one for the "current"
> to
> > > > pair
> > > > >> > with
> > > > >> > > > the leaderId. I will update the wiki page.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > Hmm, it's kind of weird to bump up the leader epoch before the
> > new
> > > > >> leader
> > > > >> > > is actually elected, right.
> > > > >> > >
> > > > >> > >
> > > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords
> > API
> > > to
> > > > >> > > indicate
> > > > >> > > > > that the follower has fetched from an invalid offset and
> > > should
> > > > >> > > truncate
> > > > >> > > > to
> > > > >> > > > > the offset/epoch indicated in the response." Observers
> can't
> > > > >> truncate
> > > > >> > > > their
> > > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > I'm not sure if I understand your question? Observers should
> > > still
> > > > >> be
> > > > >> > > able
> > > > >> > > > to truncate their logs as well.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > Hmm, I thought only the quorum nodes have local logs and
> > observers
> > > > >> don't?
> > > > >> > >
> > > > >> > > > 107. "The leader will continue sending BeginQuorumEpoch to
> > each
> > > > >> known
> > > > >> > > > voter
> > > > >> > > > > until it has received its endorsement." If a voter is down
> > > for a
> > > > >> long
> > > > >> > > > time,
> > > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary
> overhead.
> > > > >> > Similarly,
> > > > >> > > > if a
> > > > >> > > > > follower stops sending FetchQuorumRecords, does the leader
> > > keep
> > > > >> > sending
> > > > >> > > > > BeginQuorumEpoch?
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > > >> > begin-quorum-epoch
> > > > >> > > > request is for voters to quickly get the new leader
> > information;
> > > > >> > however
> > > > >> > > > even if they do not get them they can still eventually learn
> > > about
> > > > >> that
> > > > >> > > > from others via gossiping FindQuorum. I think we can adjust
> > the
> > > > >> logic
> > > > >> > to
> > > > >> > > > e.g. exponential back-off or with a limited num.retries.
> > > > >> > > >
> > > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > > >> FetchQuorumRecords
> > > > >> > > > already, it means that follower already knows that the
> broker
> > is
> > > > the
> > > > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch;
> > however
> > > > it
> > > > >> is
> > > > >> > > > possible that after a follower sends FetchQuorumRecords
> > already,
> > > > >> > suddenly
> > > > >> > > > it stops send it (possibly because it learned about a higher
> > > epoch
> > > > >> > > leader),
> > > > >> > > > and hence this broker may be a "zombie" leader and we
> propose
> > to
> > > > use
> > > > >> > the
> > > > >> > > > fetch.timeout to let the leader to try to verify if it has
> > > already
> > > > >> been
> > > > >> > > > stale.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > It just seems that we should handle these two cases in a
> > > consistent
> > > > >> way?
> > > > >> > >
> > > > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords
> from a
> > > > >> follower
> > > > >> > could mean that we no longer needs to send BeginQuorumEpoch
> > anymore
> > > > ---
> > > > >> and
> > > > >> > it is already part of our current implementations in
> > > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > > >> >
> > > > >> >
> > > > >> > > Thanks,
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > > >
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > >> >
> > > > >> > > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hello Leonard,
> > > > >> > > > > >
> > > > >> > > > > > Thanks for your comments, I'm relying in line below:
> > > > >> > > > > >
> > > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > > > >> > wge@confluent.io>
> > > > >> > > > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Hi Kafka developers,
> > > > >> > > > > > >
> > > > >> > > > > > > It's great to see this proposal and it took me some
> time
> > > to
> > > > >> > finish
> > > > >> > > > > > reading
> > > > >> > > > > > > it.
> > > > >> > > > > > >
> > > > >> > > > > > > And I have the following questions about the Proposal:
> > > > >> > > > > > >
> > > > >> > > > > > >    - How do we plan to test this design to ensure its
> > > > >> > correctness?
> > > > >> > > Or
> > > > >> > > > > > more
> > > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’ based
> > > model
> > > > >> is
> > > > >> > > > > > functional
> > > > >> > > > > > > and
> > > > >> > > > > > >    correct given that it is different from the
> original
> > > RAFT
> > > > >> > > > > > implementation
> > > > >> > > > > > >    which has formal proof of correctness?
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > > We have two planned verifications on the correctness and
> > > > >> liveness
> > > > >> > of
> > > > >> > > > the
> > > > >> > > > > > design. One is via model verification (TLA+)
> > > > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > > > >> > > > > >
> > > > >> > > > > > Another is via the concurrent simulation tests
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > >> > > > > >
> > > > >> > > > > >    - Have we considered any sensible defaults for the
> > > > >> > configuration,
> > > > >> > > > i.e.
> > > > >> > > > > > >    all the election timeout, fetch time out, etc.? Or
> we
> > > > want
> > > > >> to
> > > > >> > > > leave
> > > > >> > > > > > > this to
> > > > >> > > > > > >    a later stage when we do the performance testing,
> > etc.
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > > This is a good question, the reason we did not set any
> > > default
> > > > >> > values
> > > > >> > > > for
> > > > >> > > > > > the timeout configurations is that we think it may take
> > some
> > > > >> > > > benchmarking
> > > > >> > > > > > experiments to get these defaults right. Some high-level
> > > > >> principles
> > > > >> > > to
> > > > >> > > > > > consider: 1) the fetch.timeout should be around the same
> > > scale
> > > > >> with
> > > > >> > > zk
> > > > >> > > > > > session timeout, which is now 18 seconds by default --
> in
> > > > >> practice
> > > > >> > > > we've
> > > > >> > > > > > seen unstable networks having more than 10 secs of
> > transient
> > > > >> > > > > connectivity,
> > > > >> > > > > > 2) the election.timeout, however, should be smaller than
> > the
> > > > >> fetch
> > > > >> > > > > timeout
> > > > >> > > > > > as is also suggested as a practical optimization in
> > > > literature:
> > > > >> > > > > >
> > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > >> > > > > >
> > > > >> > > > > > Some more discussions can be found here:
> > > > >> > > > > >
> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > >    - Have we considered piggybacking
> `BeginQuorumEpoch`
> > > with
> > > > >> the
> > > > >> > `
> > > > >> > > > > > >    FetchQuorumRecords`? I might be missing something
> > > obvious
> > > > >> but
> > > > >> > I
> > > > >> > > am
> > > > >> > > > > > just
> > > > >> > > > > > >    wondering why don’t we just use the `FindQuorum`
> and
> > > > >> > > > > > > `FetchQuorumRecords`
> > > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader ->
> > other
> > > > >> voter
> > > > >> > > > > > followers, while FindQuorum / Fetch are sent from
> follower
> > > to
> > > > >> > leader.
> > > > >> > > > > > Arguably one can eventually realize the new leader and
> > epoch
> > > > via
> > > > >> > > > > gossiping
> > > > >> > > > > > FindQuorum, but that could in practice require a long
> > delay.
> > > > >> > Having a
> > > > >> > > > > > leader -> other voters request helps the new leader
> epoch
> > to
> > > > be
> > > > >> > > > > propagated
> > > > >> > > > > > faster under a pull model.
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > >    - And about the `FetchQuorumRecords` response
> schema,
> > > in
> > > > >> the
> > > > >> > > > > `Records`
> > > > >> > > > > > >    field of the response, is it just one record or all
> > the
> > > > >> > records
> > > > >> > > > > > starting
> > > > >> > > > > > >    from the FetchOffset? It seems a lot more efficient
> > if
> > > we
> > > > >> sent
> > > > >> > > all
> > > > >> > > > > the
> > > > >> > > > > > >    records during the bootstrapping of the brokers.
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> > > starting
> > > > >> > offset
> > > > >> > > of
> > > > >> > > > > the
> > > > >> > > > > > batch of records.
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > >    - Regarding the disruptive broker issues, does our
> > pull
> > > > >> based
> > > > >> > > > model
> > > > >> > > > > > >    suffer from it? If so, have we considered the
> > Pre-Vote
> > > > >> stage?
> > > > >> > If
> > > > >> > > > > not,
> > > > >> > > > > > > why?
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > The disruptive broker is stated in the original Raft
> paper
> > > > >> which is
> > > > >> > > the
> > > > >> > > > > > result of the push model design. Our analysis showed
> that
> > > with
> > > > >> the
> > > > >> > > pull
> > > > >> > > > > > model it is no longer an issue.
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > > Thanks a lot for putting this up, and I hope that my
> > > > questions
> > > > >> > can
> > > > >> > > be
> > > > >> > > > > of
> > > > >> > > > > > > some value to make this KIP better.
> > > > >> > > > > > >
> > > > >> > > > > > > Hope to hear from you soon!
> > > > >> > > > > > >
> > > > >> > > > > > > Best wishes,
> > > > >> > > > > > > Leonard
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > > >> cmccabe@apache.org
> > > > >> > >
> > > > >> > > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hi Jason,
> > > > >> > > > > > > >
> > > > >> > > > > > > > It's amazing to see this coming together :)
> > > > >> > > > > > > >
> > > > >> > > > > > > > I haven't had a chance to read in detail, but I read
> > the
> > > > >> > outline
> > > > >> > > > and
> > > > >> > > > > a
> > > > >> > > > > > > few
> > > > >> > > > > > > > things jumped out at me.
> > > > >> > > > > > > >
> > > > >> > > > > > > > First, for every epoch that is 32 bits rather than
> > 64, I
> > > > >> sort
> > > > >> > of
> > > > >> > > > > wonder
> > > > >> > > > > > > if
> > > > >> > > > > > > > that's a good long-term choice.  I keep reading
> about
> > > > stuff
> > > > >> > like
> > > > >> > > > > this:
> > > > >> > > > > > > >
> https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> > .
> > > > >> > > Obviously,
> > > > >> > > > > > that
> > > > >> > > > > > > > JIRA is about zxid, which increments much faster
> than
> > we
> > > > >> expect
> > > > >> > > > these
> > > > >> > > > > > > > leader epochs to, but it would still be good to see
> > some
> > > > >> rough
> > > > >> > > > > > > calculations
> > > > >> > > > > > > > about how long 32 bits (or really, 31 bits) will
> last
> > us
> > > > in
> > > > >> the
> > > > >> > > > cases
> > > > >> > > > > > > where
> > > > >> > > > > > > > we're using it, and what the space savings we're
> > getting
> > > > >> really
> > > > >> > > is.
> > > > >> > > > > It
> > > > >> > > > > > > > seems like in most cases the tradeoff may not be
> worth
> > > it?
> > > > >> > > > > > > >
> > > > >> > > > > > > > Another thing I've been thinking about is how we do
> > > > >> > > > bootstrapping.  I
> > > > >> > > > > > > > would prefer to be in a world where formatting a new
> > > Kafka
> > > > >> node
> > > > >> > > > was a
> > > > >> > > > > > > first
> > > > >> > > > > > > > class operation explicitly initiated by the admin,
> > > rather
> > > > >> than
> > > > >> > > > > > something
> > > > >> > > > > > > > that happened implicitly when you started up the
> > broker
> > > > and
> > > > >> > > things
> > > > >> > > > > > > "looked
> > > > >> > > > > > > > blank."
> > > > >> > > > > > > >
> > > > >> > > > > > > > The first problem is that things can "look blank"
> > > > >> accidentally
> > > > >> > if
> > > > >> > > > the
> > > > >> > > > > > > > storage system is having a bad day.  Clearly in the
> > > > non-Raft
> > > > >> > > world,
> > > > >> > > > > > this
> > > > >> > > > > > > > leads to data loss if the broker that is (re)started
> > > this
> > > > >> way
> > > > >> > was
> > > > >> > > > the
> > > > >> > > > > > > > leader for some partitions.
> > > > >> > > > > > > >
> > > > >> > > > > > > > The second problem is that we have a bit of a
> chicken
> > > and
> > > > >> egg
> > > > >> > > > problem
> > > > >> > > > > > > with
> > > > >> > > > > > > > certain configuration keys.  For example, maybe you
> > want
> > > > to
> > > > >> > > > configure
> > > > >> > > > > > > some
> > > > >> > > > > > > > connection security settings in your cluster, but
> you
> > > > don't
> > > > >> > want
> > > > >> > > > them
> > > > >> > > > > > to
> > > > >> > > > > > > > ever be stored in a plaintext config file.  (For
> > > example,
> > > > >> SCRAM
> > > > >> > > > > > > passwords,
> > > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > > configuration,
> > > > >> but
> > > > >> > > > that
> > > > >> > > > > > > brings
> > > > >> > > > > > > > up the chicken and egg problem.  The broker needs to
> > be
> > > > >> > > configured
> > > > >> > > > to
> > > > >> > > > > > > know
> > > > >> > > > > > > > how to talk to you, but you need to configure it
> > before
> > > > you
> > > > >> can
> > > > >> > > > talk
> > > > >> > > > > to
> > > > >> > > > > > > > it.  Using an external secret manager like Vault is
> > one
> > > > way
> > > > >> to
> > > > >> > > > solve
> > > > >> > > > > > > this,
> > > > >> > > > > > > > but not everyone uses an external secret manager.
> > > > >> > > > > > > >
> > > > >> > > > > > > > quorum.voters seems like a similar configuration
> key.
> > > In
> > > > >> the
> > > > >> > > > current
> > > > >> > > > > > > KIP,
> > > > >> > > > > > > > this is only read if there is no other configuration
> > > > >> specifying
> > > > >> > > the
> > > > >> > > > > > > quorum
> > > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> > wouldn't
> > > > need
> > > > >> > this
> > > > >> > > > key
> > > > >> > > > > > > > because we could assume that there was always quorum
> > > > >> > information
> > > > >> > > > > stored
> > > > >> > > > > > > > locally.
> > > > >> > > > > > > >
> > > > >> > > > > > > > best,
> > > > >> > > > > > > > Colin
> > > > >> > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson
> wrote:
> > > > >> > > > > > > > > Hi All,
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > >> > > > > > > > .
> > > > >> > > > > > > > > This proposal specifies a Raft protocol to
> > ultimately
> > > > >> replace
> > > > >> > > > > > Zookeeper
> > > > >> > > > > > > > > as
> > > > >> > > > > > > > > documented in KIP-500. Please take a look and
> share
> > > your
> > > > >> > > > thoughts.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > A few minor notes to set the stage a little bit:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > - This KIP does not specify the structure of the
> > > > messages
> > > > >> > used
> > > > >> > > to
> > > > >> > > > > > > > represent
> > > > >> > > > > > > > > metadata in Kafka, nor does it specify the
> internal
> > > API
> > > > >> that
> > > > >> > > will
> > > > >> > > > > be
> > > > >> > > > > > > used
> > > > >> > > > > > > > > by the controller. Expect these to come in later
> > > > >> proposals.
> > > > >> > > Here
> > > > >> > > > we
> > > > >> > > > > > are
> > > > >> > > > > > > > > primarily concerned with the replication protocol
> > and
> > > > >> basic
> > > > >> > > > > > operational
> > > > >> > > > > > > > > mechanics.
> > > > >> > > > > > > > > - We expect many details to change as we get
> closer
> > to
> > > > >> > > > integration
> > > > >> > > > > > with
> > > > >> > > > > > > > > the controller. Any changes we make will be made
> > > either
> > > > as
> > > > >> > > > > amendments
> > > > >> > > > > > > to
> > > > >> > > > > > > > > this KIP or, in the case of larger changes, as new
> > > > >> proposals.
> > > > >> > > > > > > > > - We have a prototype implementation which I will
> > put
> > > > >> online
> > > > >> > > > within
> > > > >> > > > > > the
> > > > >> > > > > > > > > next week which may help in understanding some
> > > details.
> > > > It
> > > > >> > has
> > > > >> > > > > > > diverged a
> > > > >> > > > > > > > > little bit from our proposal, so I am taking a
> > little
> > > > >> time to
> > > > >> > > > bring
> > > > >> > > > > > it
> > > > >> > > > > > > in
> > > > >> > > > > > > > > line. I'll post an update to this thread when it
> is
> > > > >> available
> > > > >> > > for
> > > > >> > > > > > > review.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Finally, I want to mention that this proposal was
> > > > drafted
> > > > >> by
> > > > >> > > > > myself,
> > > > >> > > > > > > > Boyang
> > > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Thanks,
> > > > >> > > > > > > > > Jason
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > --
> > > > >> > > > > > > Leonard Ge
> > > > >> > > > > > > Software Engineer Intern - Confluent
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > --
> > > > >> > > > > > -- Guozhang
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > --
> > > > >> > > > -- Guozhang
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > --
> > > > >> > -- Guozhang
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
@Jun Rao <ju...@gmail.com>

Regarding your comment about log compaction. After some deep-diving into
this we've decided to propose a new snapshot-based log cleaning mechanism
which would be used to replace the current compaction mechanism for this
meta log. A new KIP will be proposed specifically for this idea.

All,

I've updated the KIP wiki a bit updating one config "election.jitter.max.ms"
to "election.backoff.max.ms" to make it more clear about the usage: the
configured value will be the upper bound of the binary exponential backoff
time after a failed election, before starting a new one.



Guozhang



On Fri, Jun 12, 2020 at 9:26 AM Boyang Chen <re...@gmail.com>
wrote:

> Thanks for the suggestions Guozhang.
>
> On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hello Boyang,
> >
> > Thanks for the updated information. A few questions here:
> >
> > 1) Should the quorum-file also update to support multi-raft?
> >
> > I'm neutral about this, as we don't know yet how the multi-raft modules
> would behave. If
> we have different threads operating different raft groups, consolidating
> the `checkpoint` files seems
> not reasonable. We could always add `multi-quorum-file` later if possible.
>
> 2) In the previous proposal, there's fields in the FetchQuorumRecords like
> > latestDirtyOffset, is that dropped intentionally?
> >
> > I dropped the latestDirtyOffset since it is associated with the log
> compaction discussion. This is beyond this KIP scope and we could
> potentially get a separate KIP to talk about it.
>
>
> > 3) I think we also need to elaborate a bit more details regarding when to
> > send metadata request and discover-brokers; currently we only discussed
> > during bootstrap how these requests would be sent. I think the following
> > scenarios would also need these requests
> >
> > 3.a) As long as a broker does not know the current quorum (including the
> > leader and the voters), it should continue periodically ask other brokers
> > via "metadata.
> > 3.b) As long as a broker does not know all the current quorum voter's
> > connections, it should continue periodically ask other brokers via
> > "discover-brokers".
> > 3.c) When the leader's fetch timeout elapsed, it should send metadata
> > request.
> >
> > Make sense, will add to the KIP.
>
> >
> > Guozhang
> >
> >
> > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> > > Hey all,
> > >
> > > follow-up on the previous email, we made some more updates:
> > >
> > > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> > multi-raft.
> > >
> > > 2. We add observer status into the DescribeQuorumResponse as we see it
> > is a
> > > low hanging fruit which is very useful for user debugging and
> > reassignment.
> > >
> > > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which is
> > purely
> > > in charge of discovering broker connections in a gossip manner. The
> > quorum
> > > leader discovery is piggy-back on the Metadata RPC for the topic
> > partition
> > > leader, which in our case is the single metadata partition for the
> > version
> > > one.
> > >
> > > Let me know if you have any questions.
> > >
> > > Boyang
> > >
> > > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <
> reluctanthero104@gmail.com>
> > > wrote:
> > >
> > > > Hey all,
> > > >
> > > > Thanks for the great discussions so far. I'm posting some KIP updates
> > > from
> > > > our working group discussion:
> > > >
> > > > 1. We will be changing the core RPCs from single-raft API to
> > multi-raft.
> > > > This means all protocols will be "batch" in the first version, but
> the
> > > KIP
> > > > itself only illustrates the design for a single metadata topic
> > partition.
> > > > The reason is to "keep the door open" for future extensions of this
> > piece
> > > > of module such as a sharded controller or general quorum based topic
> > > > replication, beyond the current Kafka replication protocol.
> > > >
> > > > 2. We will piggy-back on the current Kafka Fetch API instead of
> > inventing
> > > > a new FetchQuorumRecords RPC. The motivation is about the same as #1
> as
> > > > well as making the integration work easier, instead of letting two
> > > similar
> > > > RPCs diverge.
> > > >
> > > > 3. In the EndQuorumEpoch protocol, instead of only sending the
> request
> > to
> > > > the most caught-up voter, we shall broadcast the information to all
> > > voters,
> > > > with a sorted voter list in descending order of their corresponding
> > > > replicated offset. In this way, the top voter will become a candidate
> > > > immediately, while the other voters shall wait for an exponential
> > > back-off
> > > > to trigger elections, which helps ensure the top voter gets elected,
> > and
> > > > the election eventually happens when the top voter is not responsive.
> > > >
> > > > Please see the updated KIP and post any questions or concerns on the
> > > > mailing thread.
> > > >
> > > > Boyang
> > > >
> > > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > >> Hi, Guozhang and Jason,
> > > >>
> > > >> Thanks for the reply. A couple of more replies.
> > > >>
> > > >> 102. Still not sure about this. How is the tombstone issue addressed
> > in
> > > >> the
> > > >> non-voter and the observer.  They can die at any point and restart
> at
> > an
> > > >> arbitrary later time, and the advancing of the firstDirty offset and
> > the
> > > >> removal of the tombstone can happen independently.
> > > >>
> > > >> 106. I agree that it would be less confusing if we used "epoch"
> > instead
> > > of
> > > >> "leader epoch" consistently.
> > > >>
> > > >> Jun
> > > >>
> > > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >>
> > > >> > Thanks Jun. Further replies are in-lined.
> > > >> >
> > > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:
> > > >> >
> > > >> > > Hi, Guozhang,
> > > >> > >
> > > >> > > Thanks for the reply. A few more replies inlined below.
> > > >> > >
> > > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <
> wangguoz@gmail.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Hello Jun,
> > > >> > > >
> > > >> > > > Thanks for your comments! I'm replying inline below:
> > > >> > > >
> > > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io>
> > wrote:
> > > >> > > >
> > > >> > > > > 101. Bootstrapping related issues.
> > > >> > > > > 101.1 Currently, we support auto broker id generation. Is
> this
> > > >> > > supported
> > > >> > > > > for bootstrap brokers?
> > > >> > > > >
> > > >> > > >
> > > >> > > > The vote ids would just be the broker ids. "bootstrap.servers"
> > > >> would be
> > > >> > > > similar to what client configs have today, where
> "quorum.voters"
> > > >> would
> > > >> > be
> > > >> > > > pre-defined config values.
> > > >> > > >
> > > >> > > >
> > > >> > > My question was on the auto generated broker id. Currently, the
> > > broker
> > > >> > can
> > > >> > > choose to have its broker Id auto generated. The generation is
> > done
> > > >> > through
> > > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how the
> > > broker
> > > >> id
> > > >> > is
> > > >> > > auto generated. "quorum.voters" also can't be set statically if
> > > broker
> > > >> > ids
> > > >> > > are auto generated.
> > > >> > >
> > > >> > > Jason has explained some ideas that we've discussed so far, the
> > > >> reason we
> > > >> > intentional did not include them so far is that we feel it is
> > out-side
> > > >> the
> > > >> > scope of KIP-595. Under the umbrella of KIP-500 we should
> definitely
> > > >> > address them though.
> > > >> >
> > > >> > On the high-level, our belief is that "joining a quorum" and
> > "joining
> > > >> (or
> > > >> > more specifically, registering brokers in) the cluster" would be
> > > >> > de-coupled a bit, where the former should be completed before we
> do
> > > the
> > > >> > latter. More specifically, assuming the quorum is already up and
> > > >> running,
> > > >> > after the newly started broker found the leader of the quorum it
> can
> > > >> send a
> > > >> > specific RegisterBroker request including its listener / protocol
> /
> > > etc,
> > > >> > and upon handling it the leader can send back the uniquely
> generated
> > > >> broker
> > > >> > id to the new broker, while also executing the "startNewBroker"
> > > >> callback as
> > > >> > the controller.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > > > 102. Log compaction. One weak spot of log compaction is for
> > the
> > > >> > > consumer
> > > >> > > > to
> > > >> > > > > deal with deletes. When a key is deleted, it's retained as a
> > > >> > tombstone
> > > >> > > > > first and then physically removed. If a client misses the
> > > >> tombstone
> > > >> > > > > (because it's physically removed), it may not be able to
> > update
> > > >> its
> > > >> > > > > metadata properly. The way we solve this in Kafka is based
> on
> > a
> > > >> > > > > configuration (log.cleaner.delete.retention.ms) and we
> > expect a
> > > >> > > consumer
> > > >> > > > > having seen an old key to finish reading the deletion
> > tombstone
> > > >> > within
> > > >> > > > that
> > > >> > > > > time. There is no strong guarantee for that since a broker
> > could
> > > >> be
> > > >> > > down
> > > >> > > > > for a long time. It would be better if we can have a more
> > > reliable
> > > >> > way
> > > >> > > of
> > > >> > > > > dealing with deletes.
> > > >> > > > >
> > > >> > > >
> > > >> > > > We propose to capture this in the "FirstDirtyOffset" field of
> > the
> > > >> > quorum
> > > >> > > > record fetch response: the offset is the maximum offset that
> log
> > > >> > > compaction
> > > >> > > > has reached up to. If the follower has fetched beyond this
> > offset
> > > it
> > > >> > > means
> > > >> > > > itself is safe hence it has seen all records up to that
> offset.
> > On
> > > >> > > getting
> > > >> > > > the response, the follower can then decide if its end offset
> > > >> actually
> > > >> > > below
> > > >> > > > that dirty offset (and hence may miss some tombstones). If
> > that's
> > > >> the
> > > >> > > case:
> > > >> > > >
> > > >> > > > 1) Naively, it could re-bootstrap metadata log from the very
> > > >> beginning
> > > >> > to
> > > >> > > > catch up.
> > > >> > > > 2) During that time, it would refrain itself from answering
> > > >> > > MetadataRequest
> > > >> > > > from any clients.
> > > >> > > >
> > > >> > > >
> > > >> > > I am not sure that the "FirstDirtyOffset" field fully addresses
> > the
> > > >> > issue.
> > > >> > > Currently, the deletion tombstone is not removed immediately
> > after a
> > > >> > round
> > > >> > > of cleaning. It's removed after a delay in a subsequent round of
> > > >> > cleaning.
> > > >> > > Consider an example where a key insertion is at offset 200 and a
> > > >> deletion
> > > >> > > tombstone of the key is at 400. Initially, FirstDirtyOffset is
> at
> > > >> 300. A
> > > >> > > follower/observer fetches from offset 0  and fetches the key at
> > > offset
> > > >> > 200.
> > > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500 and
> > the
> > > >> > > tombstone at 400 is physically removed. The follower/observer
> > > >> continues
> > > >> > the
> > > >> > > fetch, but misses offset 400. It catches all the way to
> > > >> FirstDirtyOffset
> > > >> > > and declares its metadata as ready. However, its metadata could
> be
> > > >> stale
> > > >> > > since it actually misses the deletion of the key.
> > > >> > >
> > > >> > > Yeah good question, I should have put more details in my
> > explanation
> > > >> :)
> > > >> >
> > > >> > The idea is that we will adjust the log compaction for this raft
> > based
> > > >> > metadata log: before more details to be explained, since we have
> two
> > > >> types
> > > >> > of "watermarks" here, whereas in Kafka the watermark indicates
> where
> > > >> every
> > > >> > replica have replicated up to and in Raft the watermark indicates
> > > where
> > > >> the
> > > >> > majority of replicas (here only indicating voters of the quorum,
> not
> > > >> > counting observers) have replicated up to, let's call them Kafka
> > > >> watermark
> > > >> > and Raft watermark. For this special log, we would maintain both
> > > >> > watermarks.
> > > >> >
> > > >> > When log compacting on the leader, we would only compact up to the
> > > Kafka
> > > >> > watermark, i.e. if there is at least one voter who have not
> > replicated
> > > >> an
> > > >> > entry, it would not be compacted. The "dirty-offset" is the offset
> > > that
> > > >> > we've compacted up to and is communicated to other voters, and the
> > > other
> > > >> > voters would also compact up to this value --- i.e. the difference
> > > here
> > > >> is
> > > >> > that instead of letting each replica doing log compaction
> > > independently,
> > > >> > we'll have the leader to decide upon which offset to compact to,
> and
> > > >> > propagate this value to others to follow, in a more coordinated
> > > manner.
> > > >> > Also note when there are new voters joining the quorum who has not
> > > >> > replicated up to the dirty-offset, of because of other issues they
> > > >> > truncated their logs to below the dirty-offset, they'd have to
> > > >> re-bootstrap
> > > >> > from the beginning, and during this period of time the leader
> > learned
> > > >> about
> > > >> > this lagging voter would not advance the watermark (also it would
> > not
> > > >> > decrement it), and hence not compacting either, until the voter(s)
> > has
> > > >> > caught up to that dirty-offset.
> > > >> >
> > > >> > So back to your example above, before the bootstrap voter gets to
> > 300
> > > no
> > > >> > log compaction would happen on the leader; and until later when
> the
> > > >> voter
> > > >> > have got to beyond 400 and hence replicated that tombstone, the
> log
> > > >> > compaction would possibly get to that tombstone and remove it. Say
> > > >> later it
> > > >> > the leader's log compaction reaches 500, it can send this back to
> > the
> > > >> voter
> > > >> > who can then also compact locally up to 500.
> > > >> >
> > > >> >
> > > >> > > > > 105. Quorum State: In addition to VotedId, do we need the
> > epoch
> > > >> > > > > corresponding to VotedId? Over time, the same broker Id
> could
> > be
> > > >> > voted
> > > >> > > in
> > > >> > > > > different generations with different epoch.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > Hmm, this is a good point. Originally I think the
> "LeaderEpoch"
> > > >> field
> > > >> > in
> > > >> > > > that file is corresponding to the "latest known leader epoch",
> > not
> > > >> the
> > > >> > > > "current leader epoch". For example, if the current epoch is
> N,
> > > and
> > > >> > then
> > > >> > > a
> > > >> > > > vote-request with epoch N+1 is received and the voter granted
> > the
> > > >> vote
> > > >> > > for
> > > >> > > > it, then it means for this voter it knows the "latest epoch"
> is
> > N
> > > +
> > > >> 1
> > > >> > > > although it is unknown if that sending candidate will indeed
> > > become
> > > >> the
> > > >> > > new
> > > >> > > > leader (which would only be notified via begin-quorum
> request).
> > > >> > However,
> > > >> > > > when persisting the quorum state, we would encode leader-epoch
> > to
> > > >> N+1,
> > > >> > > > while the leaderId to be the older leader.
> > > >> > > >
> > > >> > > > But now thinking about this a bit more, I feel we should use
> two
> > > >> > separate
> > > >> > > > epochs, one for the "lates known" and one for the "current" to
> > > pair
> > > >> > with
> > > >> > > > the leaderId. I will update the wiki page.
> > > >> > > >
> > > >> > > >
> > > >> > > Hmm, it's kind of weird to bump up the leader epoch before the
> new
> > > >> leader
> > > >> > > is actually elected, right.
> > > >> > >
> > > >> > >
> > > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords
> API
> > to
> > > >> > > indicate
> > > >> > > > > that the follower has fetched from an invalid offset and
> > should
> > > >> > > truncate
> > > >> > > > to
> > > >> > > > > the offset/epoch indicated in the response." Observers can't
> > > >> truncate
> > > >> > > > their
> > > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > >> > > > >
> > > >> > > > >
> > > >> > > > I'm not sure if I understand your question? Observers should
> > still
> > > >> be
> > > >> > > able
> > > >> > > > to truncate their logs as well.
> > > >> > > >
> > > >> > > >
> > > >> > > Hmm, I thought only the quorum nodes have local logs and
> observers
> > > >> don't?
> > > >> > >
> > > >> > > > 107. "The leader will continue sending BeginQuorumEpoch to
> each
> > > >> known
> > > >> > > > voter
> > > >> > > > > until it has received its endorsement." If a voter is down
> > for a
> > > >> long
> > > >> > > > time,
> > > >> > > > > sending BeginQuorumEpoch seems to add unnecessary overhead.
> > > >> > Similarly,
> > > >> > > > if a
> > > >> > > > > follower stops sending FetchQuorumRecords, does the leader
> > keep
> > > >> > sending
> > > >> > > > > BeginQuorumEpoch?
> > > >> > > > >
> > > >> > > >
> > > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > > >> > begin-quorum-epoch
> > > >> > > > request is for voters to quickly get the new leader
> information;
> > > >> > however
> > > >> > > > even if they do not get them they can still eventually learn
> > about
> > > >> that
> > > >> > > > from others via gossiping FindQuorum. I think we can adjust
> the
> > > >> logic
> > > >> > to
> > > >> > > > e.g. exponential back-off or with a limited num.retries.
> > > >> > > >
> > > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > > >> FetchQuorumRecords
> > > >> > > > already, it means that follower already knows that the broker
> is
> > > the
> > > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch;
> however
> > > it
> > > >> is
> > > >> > > > possible that after a follower sends FetchQuorumRecords
> already,
> > > >> > suddenly
> > > >> > > > it stops send it (possibly because it learned about a higher
> > epoch
> > > >> > > leader),
> > > >> > > > and hence this broker may be a "zombie" leader and we propose
> to
> > > use
> > > >> > the
> > > >> > > > fetch.timeout to let the leader to try to verify if it has
> > already
> > > >> been
> > > >> > > > stale.
> > > >> > > >
> > > >> > > >
> > > >> > > It just seems that we should handle these two cases in a
> > consistent
> > > >> way?
> > > >> > >
> > > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords from a
> > > >> follower
> > > >> > could mean that we no longer needs to send BeginQuorumEpoch
> anymore
> > > ---
> > > >> and
> > > >> > it is already part of our current implementations in
> > > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > > >> >
> > > >> >
> > > >> > > Thanks,
> > > >> > >
> > > >> > > Jun
> > > >> > >
> > > >> > > >
> > > >> > > > >
> > > >> > > > > Jun
> > > >> > > > >
> > > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > > wangguoz@gmail.com
> > > >> >
> > > >> > > > wrote:
> > > >> > > > >
> > > >> > > > > > Hello Leonard,
> > > >> > > > > >
> > > >> > > > > > Thanks for your comments, I'm relying in line below:
> > > >> > > > > >
> > > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > > >> > wge@confluent.io>
> > > >> > > > > > wrote:
> > > >> > > > > >
> > > >> > > > > > > Hi Kafka developers,
> > > >> > > > > > >
> > > >> > > > > > > It's great to see this proposal and it took me some time
> > to
> > > >> > finish
> > > >> > > > > > reading
> > > >> > > > > > > it.
> > > >> > > > > > >
> > > >> > > > > > > And I have the following questions about the Proposal:
> > > >> > > > > > >
> > > >> > > > > > >    - How do we plan to test this design to ensure its
> > > >> > correctness?
> > > >> > > Or
> > > >> > > > > > more
> > > >> > > > > > >    broadly, how do we ensure that our new ‘pull’ based
> > model
> > > >> is
> > > >> > > > > > functional
> > > >> > > > > > > and
> > > >> > > > > > >    correct given that it is different from the original
> > RAFT
> > > >> > > > > > implementation
> > > >> > > > > > >    which has formal proof of correctness?
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > > > We have two planned verifications on the correctness and
> > > >> liveness
> > > >> > of
> > > >> > > > the
> > > >> > > > > > design. One is via model verification (TLA+)
> > > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > > >> > > > > >
> > > >> > > > > > Another is via the concurrent simulation tests
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > >> > > > > >
> > > >> > > > > >    - Have we considered any sensible defaults for the
> > > >> > configuration,
> > > >> > > > i.e.
> > > >> > > > > > >    all the election timeout, fetch time out, etc.? Or we
> > > want
> > > >> to
> > > >> > > > leave
> > > >> > > > > > > this to
> > > >> > > > > > >    a later stage when we do the performance testing,
> etc.
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > > > This is a good question, the reason we did not set any
> > default
> > > >> > values
> > > >> > > > for
> > > >> > > > > > the timeout configurations is that we think it may take
> some
> > > >> > > > benchmarking
> > > >> > > > > > experiments to get these defaults right. Some high-level
> > > >> principles
> > > >> > > to
> > > >> > > > > > consider: 1) the fetch.timeout should be around the same
> > scale
> > > >> with
> > > >> > > zk
> > > >> > > > > > session timeout, which is now 18 seconds by default -- in
> > > >> practice
> > > >> > > > we've
> > > >> > > > > > seen unstable networks having more than 10 secs of
> transient
> > > >> > > > > connectivity,
> > > >> > > > > > 2) the election.timeout, however, should be smaller than
> the
> > > >> fetch
> > > >> > > > > timeout
> > > >> > > > > > as is also suggested as a practical optimization in
> > > literature:
> > > >> > > > > >
> > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > >> > > > > >
> > > >> > > > > > Some more discussions can be found here:
> > > >> > > > > >
> > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > >    - Have we considered piggybacking `BeginQuorumEpoch`
> > with
> > > >> the
> > > >> > `
> > > >> > > > > > >    FetchQuorumRecords`? I might be missing something
> > obvious
> > > >> but
> > > >> > I
> > > >> > > am
> > > >> > > > > > just
> > > >> > > > > > >    wondering why don’t we just use the `FindQuorum` and
> > > >> > > > > > > `FetchQuorumRecords`
> > > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader ->
> other
> > > >> voter
> > > >> > > > > > followers, while FindQuorum / Fetch are sent from follower
> > to
> > > >> > leader.
> > > >> > > > > > Arguably one can eventually realize the new leader and
> epoch
> > > via
> > > >> > > > > gossiping
> > > >> > > > > > FindQuorum, but that could in practice require a long
> delay.
> > > >> > Having a
> > > >> > > > > > leader -> other voters request helps the new leader epoch
> to
> > > be
> > > >> > > > > propagated
> > > >> > > > > > faster under a pull model.
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > >    - And about the `FetchQuorumRecords` response schema,
> > in
> > > >> the
> > > >> > > > > `Records`
> > > >> > > > > > >    field of the response, is it just one record or all
> the
> > > >> > records
> > > >> > > > > > starting
> > > >> > > > > > >    from the FetchOffset? It seems a lot more efficient
> if
> > we
> > > >> sent
> > > >> > > all
> > > >> > > > > the
> > > >> > > > > > >    records during the bootstrapping of the brokers.
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> > starting
> > > >> > offset
> > > >> > > of
> > > >> > > > > the
> > > >> > > > > > batch of records.
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > >    - Regarding the disruptive broker issues, does our
> pull
> > > >> based
> > > >> > > > model
> > > >> > > > > > >    suffer from it? If so, have we considered the
> Pre-Vote
> > > >> stage?
> > > >> > If
> > > >> > > > > not,
> > > >> > > > > > > why?
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > The disruptive broker is stated in the original Raft paper
> > > >> which is
> > > >> > > the
> > > >> > > > > > result of the push model design. Our analysis showed that
> > with
> > > >> the
> > > >> > > pull
> > > >> > > > > > model it is no longer an issue.
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > > Thanks a lot for putting this up, and I hope that my
> > > questions
> > > >> > can
> > > >> > > be
> > > >> > > > > of
> > > >> > > > > > > some value to make this KIP better.
> > > >> > > > > > >
> > > >> > > > > > > Hope to hear from you soon!
> > > >> > > > > > >
> > > >> > > > > > > Best wishes,
> > > >> > > > > > > Leonard
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > > >> cmccabe@apache.org
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hi Jason,
> > > >> > > > > > > >
> > > >> > > > > > > > It's amazing to see this coming together :)
> > > >> > > > > > > >
> > > >> > > > > > > > I haven't had a chance to read in detail, but I read
> the
> > > >> > outline
> > > >> > > > and
> > > >> > > > > a
> > > >> > > > > > > few
> > > >> > > > > > > > things jumped out at me.
> > > >> > > > > > > >
> > > >> > > > > > > > First, for every epoch that is 32 bits rather than
> 64, I
> > > >> sort
> > > >> > of
> > > >> > > > > wonder
> > > >> > > > > > > if
> > > >> > > > > > > > that's a good long-term choice.  I keep reading about
> > > stuff
> > > >> > like
> > > >> > > > > this:
> > > >> > > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277
> .
> > > >> > > Obviously,
> > > >> > > > > > that
> > > >> > > > > > > > JIRA is about zxid, which increments much faster than
> we
> > > >> expect
> > > >> > > > these
> > > >> > > > > > > > leader epochs to, but it would still be good to see
> some
> > > >> rough
> > > >> > > > > > > calculations
> > > >> > > > > > > > about how long 32 bits (or really, 31 bits) will last
> us
> > > in
> > > >> the
> > > >> > > > cases
> > > >> > > > > > > where
> > > >> > > > > > > > we're using it, and what the space savings we're
> getting
> > > >> really
> > > >> > > is.
> > > >> > > > > It
> > > >> > > > > > > > seems like in most cases the tradeoff may not be worth
> > it?
> > > >> > > > > > > >
> > > >> > > > > > > > Another thing I've been thinking about is how we do
> > > >> > > > bootstrapping.  I
> > > >> > > > > > > > would prefer to be in a world where formatting a new
> > Kafka
> > > >> node
> > > >> > > > was a
> > > >> > > > > > > first
> > > >> > > > > > > > class operation explicitly initiated by the admin,
> > rather
> > > >> than
> > > >> > > > > > something
> > > >> > > > > > > > that happened implicitly when you started up the
> broker
> > > and
> > > >> > > things
> > > >> > > > > > > "looked
> > > >> > > > > > > > blank."
> > > >> > > > > > > >
> > > >> > > > > > > > The first problem is that things can "look blank"
> > > >> accidentally
> > > >> > if
> > > >> > > > the
> > > >> > > > > > > > storage system is having a bad day.  Clearly in the
> > > non-Raft
> > > >> > > world,
> > > >> > > > > > this
> > > >> > > > > > > > leads to data loss if the broker that is (re)started
> > this
> > > >> way
> > > >> > was
> > > >> > > > the
> > > >> > > > > > > > leader for some partitions.
> > > >> > > > > > > >
> > > >> > > > > > > > The second problem is that we have a bit of a chicken
> > and
> > > >> egg
> > > >> > > > problem
> > > >> > > > > > > with
> > > >> > > > > > > > certain configuration keys.  For example, maybe you
> want
> > > to
> > > >> > > > configure
> > > >> > > > > > > some
> > > >> > > > > > > > connection security settings in your cluster, but you
> > > don't
> > > >> > want
> > > >> > > > them
> > > >> > > > > > to
> > > >> > > > > > > > ever be stored in a plaintext config file.  (For
> > example,
> > > >> SCRAM
> > > >> > > > > > > passwords,
> > > >> > > > > > > > etc.)  You could use a broker API to set the
> > > configuration,
> > > >> but
> > > >> > > > that
> > > >> > > > > > > brings
> > > >> > > > > > > > up the chicken and egg problem.  The broker needs to
> be
> > > >> > > configured
> > > >> > > > to
> > > >> > > > > > > know
> > > >> > > > > > > > how to talk to you, but you need to configure it
> before
> > > you
> > > >> can
> > > >> > > > talk
> > > >> > > > > to
> > > >> > > > > > > > it.  Using an external secret manager like Vault is
> one
> > > way
> > > >> to
> > > >> > > > solve
> > > >> > > > > > > this,
> > > >> > > > > > > > but not everyone uses an external secret manager.
> > > >> > > > > > > >
> > > >> > > > > > > > quorum.voters seems like a similar configuration key.
> > In
> > > >> the
> > > >> > > > current
> > > >> > > > > > > KIP,
> > > >> > > > > > > > this is only read if there is no other configuration
> > > >> specifying
> > > >> > > the
> > > >> > > > > > > quorum
> > > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we
> wouldn't
> > > need
> > > >> > this
> > > >> > > > key
> > > >> > > > > > > > because we could assume that there was always quorum
> > > >> > information
> > > >> > > > > stored
> > > >> > > > > > > > locally.
> > > >> > > > > > > >
> > > >> > > > > > > > best,
> > > >> > > > > > > > Colin
> > > >> > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > >> > > > > > > > > Hi All,
> > > >> > > > > > > > >
> > > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > >> > > > > > > > .
> > > >> > > > > > > > > This proposal specifies a Raft protocol to
> ultimately
> > > >> replace
> > > >> > > > > > Zookeeper
> > > >> > > > > > > > > as
> > > >> > > > > > > > > documented in KIP-500. Please take a look and share
> > your
> > > >> > > > thoughts.
> > > >> > > > > > > > >
> > > >> > > > > > > > > A few minor notes to set the stage a little bit:
> > > >> > > > > > > > >
> > > >> > > > > > > > > - This KIP does not specify the structure of the
> > > messages
> > > >> > used
> > > >> > > to
> > > >> > > > > > > > represent
> > > >> > > > > > > > > metadata in Kafka, nor does it specify the internal
> > API
> > > >> that
> > > >> > > will
> > > >> > > > > be
> > > >> > > > > > > used
> > > >> > > > > > > > > by the controller. Expect these to come in later
> > > >> proposals.
> > > >> > > Here
> > > >> > > > we
> > > >> > > > > > are
> > > >> > > > > > > > > primarily concerned with the replication protocol
> and
> > > >> basic
> > > >> > > > > > operational
> > > >> > > > > > > > > mechanics.
> > > >> > > > > > > > > - We expect many details to change as we get closer
> to
> > > >> > > > integration
> > > >> > > > > > with
> > > >> > > > > > > > > the controller. Any changes we make will be made
> > either
> > > as
> > > >> > > > > amendments
> > > >> > > > > > > to
> > > >> > > > > > > > > this KIP or, in the case of larger changes, as new
> > > >> proposals.
> > > >> > > > > > > > > - We have a prototype implementation which I will
> put
> > > >> online
> > > >> > > > within
> > > >> > > > > > the
> > > >> > > > > > > > > next week which may help in understanding some
> > details.
> > > It
> > > >> > has
> > > >> > > > > > > diverged a
> > > >> > > > > > > > > little bit from our proposal, so I am taking a
> little
> > > >> time to
> > > >> > > > bring
> > > >> > > > > > it
> > > >> > > > > > > in
> > > >> > > > > > > > > line. I'll post an update to this thread when it is
> > > >> available
> > > >> > > for
> > > >> > > > > > > review.
> > > >> > > > > > > > >
> > > >> > > > > > > > > Finally, I want to mention that this proposal was
> > > drafted
> > > >> by
> > > >> > > > > myself,
> > > >> > > > > > > > Boyang
> > > >> > > > > > > > > Chen, and Guozhang Wang.
> > > >> > > > > > > > >
> > > >> > > > > > > > > Thanks,
> > > >> > > > > > > > > Jason
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > --
> > > >> > > > > > > Leonard Ge
> > > >> > > > > > > Software Engineer Intern - Confluent
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > --
> > > >> > > > > > -- Guozhang
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > --
> > > >> > > > -- Guozhang
> > > >> > > >
> > > >> > >
> > > >> >
> > > >> >
> > > >> > --
> > > >> > -- Guozhang
> > > >> >
> > > >>
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Boyang Chen <re...@gmail.com>.
Thanks for the suggestions Guozhang.

On Thu, Jun 11, 2020 at 2:51 PM Guozhang Wang <wa...@gmail.com> wrote:

> Hello Boyang,
>
> Thanks for the updated information. A few questions here:
>
> 1) Should the quorum-file also update to support multi-raft?
>
> I'm neutral about this, as we don't know yet how the multi-raft modules
would behave. If
we have different threads operating different raft groups, consolidating
the `checkpoint` files seems
not reasonable. We could always add `multi-quorum-file` later if possible.

2) In the previous proposal, there's fields in the FetchQuorumRecords like
> latestDirtyOffset, is that dropped intentionally?
>
> I dropped the latestDirtyOffset since it is associated with the log
compaction discussion. This is beyond this KIP scope and we could
potentially get a separate KIP to talk about it.


> 3) I think we also need to elaborate a bit more details regarding when to
> send metadata request and discover-brokers; currently we only discussed
> during bootstrap how these requests would be sent. I think the following
> scenarios would also need these requests
>
> 3.a) As long as a broker does not know the current quorum (including the
> leader and the voters), it should continue periodically ask other brokers
> via "metadata.
> 3.b) As long as a broker does not know all the current quorum voter's
> connections, it should continue periodically ask other brokers via
> "discover-brokers".
> 3.c) When the leader's fetch timeout elapsed, it should send metadata
> request.
>
> Make sense, will add to the KIP.

>
> Guozhang
>
>
> On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <re...@gmail.com>
> wrote:
>
> > Hey all,
> >
> > follow-up on the previous email, we made some more updates:
> >
> > 1. The Alter/DescribeQuorum RPCs are also re-structured to use
> multi-raft.
> >
> > 2. We add observer status into the DescribeQuorumResponse as we see it
> is a
> > low hanging fruit which is very useful for user debugging and
> reassignment.
> >
> > 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which is
> purely
> > in charge of discovering broker connections in a gossip manner. The
> quorum
> > leader discovery is piggy-back on the Metadata RPC for the topic
> partition
> > leader, which in our case is the single metadata partition for the
> version
> > one.
> >
> > Let me know if you have any questions.
> >
> > Boyang
> >
> > On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> > > Hey all,
> > >
> > > Thanks for the great discussions so far. I'm posting some KIP updates
> > from
> > > our working group discussion:
> > >
> > > 1. We will be changing the core RPCs from single-raft API to
> multi-raft.
> > > This means all protocols will be "batch" in the first version, but the
> > KIP
> > > itself only illustrates the design for a single metadata topic
> partition.
> > > The reason is to "keep the door open" for future extensions of this
> piece
> > > of module such as a sharded controller or general quorum based topic
> > > replication, beyond the current Kafka replication protocol.
> > >
> > > 2. We will piggy-back on the current Kafka Fetch API instead of
> inventing
> > > a new FetchQuorumRecords RPC. The motivation is about the same as #1 as
> > > well as making the integration work easier, instead of letting two
> > similar
> > > RPCs diverge.
> > >
> > > 3. In the EndQuorumEpoch protocol, instead of only sending the request
> to
> > > the most caught-up voter, we shall broadcast the information to all
> > voters,
> > > with a sorted voter list in descending order of their corresponding
> > > replicated offset. In this way, the top voter will become a candidate
> > > immediately, while the other voters shall wait for an exponential
> > back-off
> > > to trigger elections, which helps ensure the top voter gets elected,
> and
> > > the election eventually happens when the top voter is not responsive.
> > >
> > > Please see the updated KIP and post any questions or concerns on the
> > > mailing thread.
> > >
> > > Boyang
> > >
> > > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:
> > >
> > >> Hi, Guozhang and Jason,
> > >>
> > >> Thanks for the reply. A couple of more replies.
> > >>
> > >> 102. Still not sure about this. How is the tombstone issue addressed
> in
> > >> the
> > >> non-voter and the observer.  They can die at any point and restart at
> an
> > >> arbitrary later time, and the advancing of the firstDirty offset and
> the
> > >> removal of the tombstone can happen independently.
> > >>
> > >> 106. I agree that it would be less confusing if we used "epoch"
> instead
> > of
> > >> "leader epoch" consistently.
> > >>
> > >> Jun
> > >>
> > >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >>
> > >> > Thanks Jun. Further replies are in-lined.
> > >> >
> > >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > Hi, Guozhang,
> > >> > >
> > >> > > Thanks for the reply. A few more replies inlined below.
> > >> > >
> > >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > Hello Jun,
> > >> > > >
> > >> > > > Thanks for your comments! I'm replying inline below:
> > >> > > >
> > >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io>
> wrote:
> > >> > > >
> > >> > > > > 101. Bootstrapping related issues.
> > >> > > > > 101.1 Currently, we support auto broker id generation. Is this
> > >> > > supported
> > >> > > > > for bootstrap brokers?
> > >> > > > >
> > >> > > >
> > >> > > > The vote ids would just be the broker ids. "bootstrap.servers"
> > >> would be
> > >> > > > similar to what client configs have today, where "quorum.voters"
> > >> would
> > >> > be
> > >> > > > pre-defined config values.
> > >> > > >
> > >> > > >
> > >> > > My question was on the auto generated broker id. Currently, the
> > broker
> > >> > can
> > >> > > choose to have its broker Id auto generated. The generation is
> done
> > >> > through
> > >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how the
> > broker
> > >> id
> > >> > is
> > >> > > auto generated. "quorum.voters" also can't be set statically if
> > broker
> > >> > ids
> > >> > > are auto generated.
> > >> > >
> > >> > > Jason has explained some ideas that we've discussed so far, the
> > >> reason we
> > >> > intentional did not include them so far is that we feel it is
> out-side
> > >> the
> > >> > scope of KIP-595. Under the umbrella of KIP-500 we should definitely
> > >> > address them though.
> > >> >
> > >> > On the high-level, our belief is that "joining a quorum" and
> "joining
> > >> (or
> > >> > more specifically, registering brokers in) the cluster" would be
> > >> > de-coupled a bit, where the former should be completed before we do
> > the
> > >> > latter. More specifically, assuming the quorum is already up and
> > >> running,
> > >> > after the newly started broker found the leader of the quorum it can
> > >> send a
> > >> > specific RegisterBroker request including its listener / protocol /
> > etc,
> > >> > and upon handling it the leader can send back the uniquely generated
> > >> broker
> > >> > id to the new broker, while also executing the "startNewBroker"
> > >> callback as
> > >> > the controller.
> > >> >
> > >> >
> > >> > >
> > >> > > > > 102. Log compaction. One weak spot of log compaction is for
> the
> > >> > > consumer
> > >> > > > to
> > >> > > > > deal with deletes. When a key is deleted, it's retained as a
> > >> > tombstone
> > >> > > > > first and then physically removed. If a client misses the
> > >> tombstone
> > >> > > > > (because it's physically removed), it may not be able to
> update
> > >> its
> > >> > > > > metadata properly. The way we solve this in Kafka is based on
> a
> > >> > > > > configuration (log.cleaner.delete.retention.ms) and we
> expect a
> > >> > > consumer
> > >> > > > > having seen an old key to finish reading the deletion
> tombstone
> > >> > within
> > >> > > > that
> > >> > > > > time. There is no strong guarantee for that since a broker
> could
> > >> be
> > >> > > down
> > >> > > > > for a long time. It would be better if we can have a more
> > reliable
> > >> > way
> > >> > > of
> > >> > > > > dealing with deletes.
> > >> > > > >
> > >> > > >
> > >> > > > We propose to capture this in the "FirstDirtyOffset" field of
> the
> > >> > quorum
> > >> > > > record fetch response: the offset is the maximum offset that log
> > >> > > compaction
> > >> > > > has reached up to. If the follower has fetched beyond this
> offset
> > it
> > >> > > means
> > >> > > > itself is safe hence it has seen all records up to that offset.
> On
> > >> > > getting
> > >> > > > the response, the follower can then decide if its end offset
> > >> actually
> > >> > > below
> > >> > > > that dirty offset (and hence may miss some tombstones). If
> that's
> > >> the
> > >> > > case:
> > >> > > >
> > >> > > > 1) Naively, it could re-bootstrap metadata log from the very
> > >> beginning
> > >> > to
> > >> > > > catch up.
> > >> > > > 2) During that time, it would refrain itself from answering
> > >> > > MetadataRequest
> > >> > > > from any clients.
> > >> > > >
> > >> > > >
> > >> > > I am not sure that the "FirstDirtyOffset" field fully addresses
> the
> > >> > issue.
> > >> > > Currently, the deletion tombstone is not removed immediately
> after a
> > >> > round
> > >> > > of cleaning. It's removed after a delay in a subsequent round of
> > >> > cleaning.
> > >> > > Consider an example where a key insertion is at offset 200 and a
> > >> deletion
> > >> > > tombstone of the key is at 400. Initially, FirstDirtyOffset is at
> > >> 300. A
> > >> > > follower/observer fetches from offset 0  and fetches the key at
> > offset
> > >> > 200.
> > >> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500 and
> the
> > >> > > tombstone at 400 is physically removed. The follower/observer
> > >> continues
> > >> > the
> > >> > > fetch, but misses offset 400. It catches all the way to
> > >> FirstDirtyOffset
> > >> > > and declares its metadata as ready. However, its metadata could be
> > >> stale
> > >> > > since it actually misses the deletion of the key.
> > >> > >
> > >> > > Yeah good question, I should have put more details in my
> explanation
> > >> :)
> > >> >
> > >> > The idea is that we will adjust the log compaction for this raft
> based
> > >> > metadata log: before more details to be explained, since we have two
> > >> types
> > >> > of "watermarks" here, whereas in Kafka the watermark indicates where
> > >> every
> > >> > replica have replicated up to and in Raft the watermark indicates
> > where
> > >> the
> > >> > majority of replicas (here only indicating voters of the quorum, not
> > >> > counting observers) have replicated up to, let's call them Kafka
> > >> watermark
> > >> > and Raft watermark. For this special log, we would maintain both
> > >> > watermarks.
> > >> >
> > >> > When log compacting on the leader, we would only compact up to the
> > Kafka
> > >> > watermark, i.e. if there is at least one voter who have not
> replicated
> > >> an
> > >> > entry, it would not be compacted. The "dirty-offset" is the offset
> > that
> > >> > we've compacted up to and is communicated to other voters, and the
> > other
> > >> > voters would also compact up to this value --- i.e. the difference
> > here
> > >> is
> > >> > that instead of letting each replica doing log compaction
> > independently,
> > >> > we'll have the leader to decide upon which offset to compact to, and
> > >> > propagate this value to others to follow, in a more coordinated
> > manner.
> > >> > Also note when there are new voters joining the quorum who has not
> > >> > replicated up to the dirty-offset, of because of other issues they
> > >> > truncated their logs to below the dirty-offset, they'd have to
> > >> re-bootstrap
> > >> > from the beginning, and during this period of time the leader
> learned
> > >> about
> > >> > this lagging voter would not advance the watermark (also it would
> not
> > >> > decrement it), and hence not compacting either, until the voter(s)
> has
> > >> > caught up to that dirty-offset.
> > >> >
> > >> > So back to your example above, before the bootstrap voter gets to
> 300
> > no
> > >> > log compaction would happen on the leader; and until later when the
> > >> voter
> > >> > have got to beyond 400 and hence replicated that tombstone, the log
> > >> > compaction would possibly get to that tombstone and remove it. Say
> > >> later it
> > >> > the leader's log compaction reaches 500, it can send this back to
> the
> > >> voter
> > >> > who can then also compact locally up to 500.
> > >> >
> > >> >
> > >> > > > > 105. Quorum State: In addition to VotedId, do we need the
> epoch
> > >> > > > > corresponding to VotedId? Over time, the same broker Id could
> be
> > >> > voted
> > >> > > in
> > >> > > > > different generations with different epoch.
> > >> > > > >
> > >> > > > >
> > >> > > > Hmm, this is a good point. Originally I think the "LeaderEpoch"
> > >> field
> > >> > in
> > >> > > > that file is corresponding to the "latest known leader epoch",
> not
> > >> the
> > >> > > > "current leader epoch". For example, if the current epoch is N,
> > and
> > >> > then
> > >> > > a
> > >> > > > vote-request with epoch N+1 is received and the voter granted
> the
> > >> vote
> > >> > > for
> > >> > > > it, then it means for this voter it knows the "latest epoch" is
> N
> > +
> > >> 1
> > >> > > > although it is unknown if that sending candidate will indeed
> > become
> > >> the
> > >> > > new
> > >> > > > leader (which would only be notified via begin-quorum request).
> > >> > However,
> > >> > > > when persisting the quorum state, we would encode leader-epoch
> to
> > >> N+1,
> > >> > > > while the leaderId to be the older leader.
> > >> > > >
> > >> > > > But now thinking about this a bit more, I feel we should use two
> > >> > separate
> > >> > > > epochs, one for the "lates known" and one for the "current" to
> > pair
> > >> > with
> > >> > > > the leaderId. I will update the wiki page.
> > >> > > >
> > >> > > >
> > >> > > Hmm, it's kind of weird to bump up the leader epoch before the new
> > >> leader
> > >> > > is actually elected, right.
> > >> > >
> > >> > >
> > >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API
> to
> > >> > > indicate
> > >> > > > > that the follower has fetched from an invalid offset and
> should
> > >> > > truncate
> > >> > > > to
> > >> > > > > the offset/epoch indicated in the response." Observers can't
> > >> truncate
> > >> > > > their
> > >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > >> > > > >
> > >> > > > >
> > >> > > > I'm not sure if I understand your question? Observers should
> still
> > >> be
> > >> > > able
> > >> > > > to truncate their logs as well.
> > >> > > >
> > >> > > >
> > >> > > Hmm, I thought only the quorum nodes have local logs and observers
> > >> don't?
> > >> > >
> > >> > > > 107. "The leader will continue sending BeginQuorumEpoch to each
> > >> known
> > >> > > > voter
> > >> > > > > until it has received its endorsement." If a voter is down
> for a
> > >> long
> > >> > > > time,
> > >> > > > > sending BeginQuorumEpoch seems to add unnecessary overhead.
> > >> > Similarly,
> > >> > > > if a
> > >> > > > > follower stops sending FetchQuorumRecords, does the leader
> keep
> > >> > sending
> > >> > > > > BeginQuorumEpoch?
> > >> > > > >
> > >> > > >
> > >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > >> > begin-quorum-epoch
> > >> > > > request is for voters to quickly get the new leader information;
> > >> > however
> > >> > > > even if they do not get them they can still eventually learn
> about
> > >> that
> > >> > > > from others via gossiping FindQuorum. I think we can adjust the
> > >> logic
> > >> > to
> > >> > > > e.g. exponential back-off or with a limited num.retries.
> > >> > > >
> > >> > > > Regarding FetchQuorumRecords: if the follower sends
> > >> FetchQuorumRecords
> > >> > > > already, it means that follower already knows that the broker is
> > the
> > >> > > > leader, and hence we can stop retrying BeginQuorumEpoch; however
> > it
> > >> is
> > >> > > > possible that after a follower sends FetchQuorumRecords already,
> > >> > suddenly
> > >> > > > it stops send it (possibly because it learned about a higher
> epoch
> > >> > > leader),
> > >> > > > and hence this broker may be a "zombie" leader and we propose to
> > use
> > >> > the
> > >> > > > fetch.timeout to let the leader to try to verify if it has
> already
> > >> been
> > >> > > > stale.
> > >> > > >
> > >> > > >
> > >> > > It just seems that we should handle these two cases in a
> consistent
> > >> way?
> > >> > >
> > >> > > Yes I agree, on the leader's side, the FetchQuorumRecords from a
> > >> follower
> > >> > could mean that we no longer needs to send BeginQuorumEpoch anymore
> > ---
> > >> and
> > >> > it is already part of our current implementations in
> > >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> > >> >
> > >> >
> > >> > > Thanks,
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > > >
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> > wangguoz@gmail.com
> > >> >
> > >> > > > wrote:
> > >> > > > >
> > >> > > > > > Hello Leonard,
> > >> > > > > >
> > >> > > > > > Thanks for your comments, I'm relying in line below:
> > >> > > > > >
> > >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > >> > wge@confluent.io>
> > >> > > > > > wrote:
> > >> > > > > >
> > >> > > > > > > Hi Kafka developers,
> > >> > > > > > >
> > >> > > > > > > It's great to see this proposal and it took me some time
> to
> > >> > finish
> > >> > > > > > reading
> > >> > > > > > > it.
> > >> > > > > > >
> > >> > > > > > > And I have the following questions about the Proposal:
> > >> > > > > > >
> > >> > > > > > >    - How do we plan to test this design to ensure its
> > >> > correctness?
> > >> > > Or
> > >> > > > > > more
> > >> > > > > > >    broadly, how do we ensure that our new ‘pull’ based
> model
> > >> is
> > >> > > > > > functional
> > >> > > > > > > and
> > >> > > > > > >    correct given that it is different from the original
> RAFT
> > >> > > > > > implementation
> > >> > > > > > >    which has formal proof of correctness?
> > >> > > > > > >
> > >> > > > > >
> > >> > > > > > We have two planned verifications on the correctness and
> > >> liveness
> > >> > of
> > >> > > > the
> > >> > > > > > design. One is via model verification (TLA+)
> > >> > > > > > https://github.com/guozhangwang/kafka-specification
> > >> > > > > >
> > >> > > > > > Another is via the concurrent simulation tests
> > >> > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > >> > > > > >
> > >> > > > > >    - Have we considered any sensible defaults for the
> > >> > configuration,
> > >> > > > i.e.
> > >> > > > > > >    all the election timeout, fetch time out, etc.? Or we
> > want
> > >> to
> > >> > > > leave
> > >> > > > > > > this to
> > >> > > > > > >    a later stage when we do the performance testing, etc.
> > >> > > > > > >
> > >> > > > > >
> > >> > > > > > This is a good question, the reason we did not set any
> default
> > >> > values
> > >> > > > for
> > >> > > > > > the timeout configurations is that we think it may take some
> > >> > > > benchmarking
> > >> > > > > > experiments to get these defaults right. Some high-level
> > >> principles
> > >> > > to
> > >> > > > > > consider: 1) the fetch.timeout should be around the same
> scale
> > >> with
> > >> > > zk
> > >> > > > > > session timeout, which is now 18 seconds by default -- in
> > >> practice
> > >> > > > we've
> > >> > > > > > seen unstable networks having more than 10 secs of transient
> > >> > > > > connectivity,
> > >> > > > > > 2) the election.timeout, however, should be smaller than the
> > >> fetch
> > >> > > > > timeout
> > >> > > > > > as is also suggested as a practical optimization in
> > literature:
> > >> > > > > >
> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > >> > > > > >
> > >> > > > > > Some more discussions can be found here:
> > >> > > > > >
> > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > >    - Have we considered piggybacking `BeginQuorumEpoch`
> with
> > >> the
> > >> > `
> > >> > > > > > >    FetchQuorumRecords`? I might be missing something
> obvious
> > >> but
> > >> > I
> > >> > > am
> > >> > > > > > just
> > >> > > > > > >    wondering why don’t we just use the `FindQuorum` and
> > >> > > > > > > `FetchQuorumRecords`
> > >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > >> > > > > > >
> > >> > > > > >
> > >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader -> other
> > >> voter
> > >> > > > > > followers, while FindQuorum / Fetch are sent from follower
> to
> > >> > leader.
> > >> > > > > > Arguably one can eventually realize the new leader and epoch
> > via
> > >> > > > > gossiping
> > >> > > > > > FindQuorum, but that could in practice require a long delay.
> > >> > Having a
> > >> > > > > > leader -> other voters request helps the new leader epoch to
> > be
> > >> > > > > propagated
> > >> > > > > > faster under a pull model.
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > >    - And about the `FetchQuorumRecords` response schema,
> in
> > >> the
> > >> > > > > `Records`
> > >> > > > > > >    field of the response, is it just one record or all the
> > >> > records
> > >> > > > > > starting
> > >> > > > > > >    from the FetchOffset? It seems a lot more efficient if
> we
> > >> sent
> > >> > > all
> > >> > > > > the
> > >> > > > > > >    records during the bootstrapping of the brokers.
> > >> > > > > > >
> > >> > > > > >
> > >> > > > > > Yes the fetching is batched: FetchOffset is just the
> starting
> > >> > offset
> > >> > > of
> > >> > > > > the
> > >> > > > > > batch of records.
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > >    - Regarding the disruptive broker issues, does our pull
> > >> based
> > >> > > > model
> > >> > > > > > >    suffer from it? If so, have we considered the Pre-Vote
> > >> stage?
> > >> > If
> > >> > > > > not,
> > >> > > > > > > why?
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > The disruptive broker is stated in the original Raft paper
> > >> which is
> > >> > > the
> > >> > > > > > result of the push model design. Our analysis showed that
> with
> > >> the
> > >> > > pull
> > >> > > > > > model it is no longer an issue.
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > > Thanks a lot for putting this up, and I hope that my
> > questions
> > >> > can
> > >> > > be
> > >> > > > > of
> > >> > > > > > > some value to make this KIP better.
> > >> > > > > > >
> > >> > > > > > > Hope to hear from you soon!
> > >> > > > > > >
> > >> > > > > > > Best wishes,
> > >> > > > > > > Leonard
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> > >> cmccabe@apache.org
> > >> > >
> > >> > > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hi Jason,
> > >> > > > > > > >
> > >> > > > > > > > It's amazing to see this coming together :)
> > >> > > > > > > >
> > >> > > > > > > > I haven't had a chance to read in detail, but I read the
> > >> > outline
> > >> > > > and
> > >> > > > > a
> > >> > > > > > > few
> > >> > > > > > > > things jumped out at me.
> > >> > > > > > > >
> > >> > > > > > > > First, for every epoch that is 32 bits rather than 64, I
> > >> sort
> > >> > of
> > >> > > > > wonder
> > >> > > > > > > if
> > >> > > > > > > > that's a good long-term choice.  I keep reading about
> > stuff
> > >> > like
> > >> > > > > this:
> > >> > > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
> > >> > > Obviously,
> > >> > > > > > that
> > >> > > > > > > > JIRA is about zxid, which increments much faster than we
> > >> expect
> > >> > > > these
> > >> > > > > > > > leader epochs to, but it would still be good to see some
> > >> rough
> > >> > > > > > > calculations
> > >> > > > > > > > about how long 32 bits (or really, 31 bits) will last us
> > in
> > >> the
> > >> > > > cases
> > >> > > > > > > where
> > >> > > > > > > > we're using it, and what the space savings we're getting
> > >> really
> > >> > > is.
> > >> > > > > It
> > >> > > > > > > > seems like in most cases the tradeoff may not be worth
> it?
> > >> > > > > > > >
> > >> > > > > > > > Another thing I've been thinking about is how we do
> > >> > > > bootstrapping.  I
> > >> > > > > > > > would prefer to be in a world where formatting a new
> Kafka
> > >> node
> > >> > > > was a
> > >> > > > > > > first
> > >> > > > > > > > class operation explicitly initiated by the admin,
> rather
> > >> than
> > >> > > > > > something
> > >> > > > > > > > that happened implicitly when you started up the broker
> > and
> > >> > > things
> > >> > > > > > > "looked
> > >> > > > > > > > blank."
> > >> > > > > > > >
> > >> > > > > > > > The first problem is that things can "look blank"
> > >> accidentally
> > >> > if
> > >> > > > the
> > >> > > > > > > > storage system is having a bad day.  Clearly in the
> > non-Raft
> > >> > > world,
> > >> > > > > > this
> > >> > > > > > > > leads to data loss if the broker that is (re)started
> this
> > >> way
> > >> > was
> > >> > > > the
> > >> > > > > > > > leader for some partitions.
> > >> > > > > > > >
> > >> > > > > > > > The second problem is that we have a bit of a chicken
> and
> > >> egg
> > >> > > > problem
> > >> > > > > > > with
> > >> > > > > > > > certain configuration keys.  For example, maybe you want
> > to
> > >> > > > configure
> > >> > > > > > > some
> > >> > > > > > > > connection security settings in your cluster, but you
> > don't
> > >> > want
> > >> > > > them
> > >> > > > > > to
> > >> > > > > > > > ever be stored in a plaintext config file.  (For
> example,
> > >> SCRAM
> > >> > > > > > > passwords,
> > >> > > > > > > > etc.)  You could use a broker API to set the
> > configuration,
> > >> but
> > >> > > > that
> > >> > > > > > > brings
> > >> > > > > > > > up the chicken and egg problem.  The broker needs to be
> > >> > > configured
> > >> > > > to
> > >> > > > > > > know
> > >> > > > > > > > how to talk to you, but you need to configure it before
> > you
> > >> can
> > >> > > > talk
> > >> > > > > to
> > >> > > > > > > > it.  Using an external secret manager like Vault is one
> > way
> > >> to
> > >> > > > solve
> > >> > > > > > > this,
> > >> > > > > > > > but not everyone uses an external secret manager.
> > >> > > > > > > >
> > >> > > > > > > > quorum.voters seems like a similar configuration key.
> In
> > >> the
> > >> > > > current
> > >> > > > > > > KIP,
> > >> > > > > > > > this is only read if there is no other configuration
> > >> specifying
> > >> > > the
> > >> > > > > > > quorum
> > >> > > > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't
> > need
> > >> > this
> > >> > > > key
> > >> > > > > > > > because we could assume that there was always quorum
> > >> > information
> > >> > > > > stored
> > >> > > > > > > > locally.
> > >> > > > > > > >
> > >> > > > > > > > best,
> > >> > > > > > > > Colin
> > >> > > > > > > >
> > >> > > > > > > >
> > >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > >> > > > > > > > > Hi All,
> > >> > > > > > > > >
> > >> > > > > > > > > I'd like to start a discussion on KIP-595:
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > >> > > > > > > > .
> > >> > > > > > > > > This proposal specifies a Raft protocol to ultimately
> > >> replace
> > >> > > > > > Zookeeper
> > >> > > > > > > > > as
> > >> > > > > > > > > documented in KIP-500. Please take a look and share
> your
> > >> > > > thoughts.
> > >> > > > > > > > >
> > >> > > > > > > > > A few minor notes to set the stage a little bit:
> > >> > > > > > > > >
> > >> > > > > > > > > - This KIP does not specify the structure of the
> > messages
> > >> > used
> > >> > > to
> > >> > > > > > > > represent
> > >> > > > > > > > > metadata in Kafka, nor does it specify the internal
> API
> > >> that
> > >> > > will
> > >> > > > > be
> > >> > > > > > > used
> > >> > > > > > > > > by the controller. Expect these to come in later
> > >> proposals.
> > >> > > Here
> > >> > > > we
> > >> > > > > > are
> > >> > > > > > > > > primarily concerned with the replication protocol and
> > >> basic
> > >> > > > > > operational
> > >> > > > > > > > > mechanics.
> > >> > > > > > > > > - We expect many details to change as we get closer to
> > >> > > > integration
> > >> > > > > > with
> > >> > > > > > > > > the controller. Any changes we make will be made
> either
> > as
> > >> > > > > amendments
> > >> > > > > > > to
> > >> > > > > > > > > this KIP or, in the case of larger changes, as new
> > >> proposals.
> > >> > > > > > > > > - We have a prototype implementation which I will put
> > >> online
> > >> > > > within
> > >> > > > > > the
> > >> > > > > > > > > next week which may help in understanding some
> details.
> > It
> > >> > has
> > >> > > > > > > diverged a
> > >> > > > > > > > > little bit from our proposal, so I am taking a little
> > >> time to
> > >> > > > bring
> > >> > > > > > it
> > >> > > > > > > in
> > >> > > > > > > > > line. I'll post an update to this thread when it is
> > >> available
> > >> > > for
> > >> > > > > > > review.
> > >> > > > > > > > >
> > >> > > > > > > > > Finally, I want to mention that this proposal was
> > drafted
> > >> by
> > >> > > > > myself,
> > >> > > > > > > > Boyang
> > >> > > > > > > > > Chen, and Guozhang Wang.
> > >> > > > > > > > >
> > >> > > > > > > > > Thanks,
> > >> > > > > > > > > Jason
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > --
> > >> > > > > > > Leonard Ge
> > >> > > > > > > Software Engineer Intern - Confluent
> > >> > > > > > >
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > --
> > >> > > > > > -- Guozhang
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > > >
> > >> > > > --
> > >> > > > -- Guozhang
> > >> > > >
> > >> > >
> > >> >
> > >> >
> > >> > --
> > >> > -- Guozhang
> > >> >
> > >>
> > >
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Boyang,

Thanks for the updated information. A few questions here:

1) Should the quorum-file also update to support multi-raft?

2) In the previous proposal, there's fields in the FetchQuorumRecords like
latestDirtyOffset, is that dropped intentionally?

3) I think we also need to elaborate a bit more details regarding when to
send metadata request and discover-brokers; currently we only discussed
during bootstrap how these requests would be sent. I think the following
scenarios would also need these requests

3.a) As long as a broker does not know the current quorum (including the
leader and the voters), it should continue periodically ask other brokers
via "metadata.
3.b) As long as a broker does not know all the current quorum voter's
connections, it should continue periodically ask other brokers via
"discover-brokers".
3.c) When the leader's fetch timeout elapsed, it should send metadata
request.


Guozhang


On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen <re...@gmail.com>
wrote:

> Hey all,
>
> follow-up on the previous email, we made some more updates:
>
> 1. The Alter/DescribeQuorum RPCs are also re-structured to use multi-raft.
>
> 2. We add observer status into the DescribeQuorumResponse as we see it is a
> low hanging fruit which is very useful for user debugging and reassignment.
>
> 3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which is purely
> in charge of discovering broker connections in a gossip manner. The quorum
> leader discovery is piggy-back on the Metadata RPC for the topic partition
> leader, which in our case is the single metadata partition for the version
> one.
>
> Let me know if you have any questions.
>
> Boyang
>
> On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <re...@gmail.com>
> wrote:
>
> > Hey all,
> >
> > Thanks for the great discussions so far. I'm posting some KIP updates
> from
> > our working group discussion:
> >
> > 1. We will be changing the core RPCs from single-raft API to multi-raft.
> > This means all protocols will be "batch" in the first version, but the
> KIP
> > itself only illustrates the design for a single metadata topic partition.
> > The reason is to "keep the door open" for future extensions of this piece
> > of module such as a sharded controller or general quorum based topic
> > replication, beyond the current Kafka replication protocol.
> >
> > 2. We will piggy-back on the current Kafka Fetch API instead of inventing
> > a new FetchQuorumRecords RPC. The motivation is about the same as #1 as
> > well as making the integration work easier, instead of letting two
> similar
> > RPCs diverge.
> >
> > 3. In the EndQuorumEpoch protocol, instead of only sending the request to
> > the most caught-up voter, we shall broadcast the information to all
> voters,
> > with a sorted voter list in descending order of their corresponding
> > replicated offset. In this way, the top voter will become a candidate
> > immediately, while the other voters shall wait for an exponential
> back-off
> > to trigger elections, which helps ensure the top voter gets elected, and
> > the election eventually happens when the top voter is not responsive.
> >
> > Please see the updated KIP and post any questions or concerns on the
> > mailing thread.
> >
> > Boyang
> >
> > On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:
> >
> >> Hi, Guozhang and Jason,
> >>
> >> Thanks for the reply. A couple of more replies.
> >>
> >> 102. Still not sure about this. How is the tombstone issue addressed in
> >> the
> >> non-voter and the observer.  They can die at any point and restart at an
> >> arbitrary later time, and the advancing of the firstDirty offset and the
> >> removal of the tombstone can happen independently.
> >>
> >> 106. I agree that it would be less confusing if we used "epoch" instead
> of
> >> "leader epoch" consistently.
> >>
> >> Jun
> >>
> >> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >> > Thanks Jun. Further replies are in-lined.
> >> >
> >> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > Hi, Guozhang,
> >> > >
> >> > > Thanks for the reply. A few more replies inlined below.
> >> > >
> >> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >> > >
> >> > > > Hello Jun,
> >> > > >
> >> > > > Thanks for your comments! I'm replying inline below:
> >> > > >
> >> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
> >> > > >
> >> > > > > 101. Bootstrapping related issues.
> >> > > > > 101.1 Currently, we support auto broker id generation. Is this
> >> > > supported
> >> > > > > for bootstrap brokers?
> >> > > > >
> >> > > >
> >> > > > The vote ids would just be the broker ids. "bootstrap.servers"
> >> would be
> >> > > > similar to what client configs have today, where "quorum.voters"
> >> would
> >> > be
> >> > > > pre-defined config values.
> >> > > >
> >> > > >
> >> > > My question was on the auto generated broker id. Currently, the
> broker
> >> > can
> >> > > choose to have its broker Id auto generated. The generation is done
> >> > through
> >> > > ZK to guarantee uniqueness. Without ZK, it's not clear how the
> broker
> >> id
> >> > is
> >> > > auto generated. "quorum.voters" also can't be set statically if
> broker
> >> > ids
> >> > > are auto generated.
> >> > >
> >> > > Jason has explained some ideas that we've discussed so far, the
> >> reason we
> >> > intentional did not include them so far is that we feel it is out-side
> >> the
> >> > scope of KIP-595. Under the umbrella of KIP-500 we should definitely
> >> > address them though.
> >> >
> >> > On the high-level, our belief is that "joining a quorum" and "joining
> >> (or
> >> > more specifically, registering brokers in) the cluster" would be
> >> > de-coupled a bit, where the former should be completed before we do
> the
> >> > latter. More specifically, assuming the quorum is already up and
> >> running,
> >> > after the newly started broker found the leader of the quorum it can
> >> send a
> >> > specific RegisterBroker request including its listener / protocol /
> etc,
> >> > and upon handling it the leader can send back the uniquely generated
> >> broker
> >> > id to the new broker, while also executing the "startNewBroker"
> >> callback as
> >> > the controller.
> >> >
> >> >
> >> > >
> >> > > > > 102. Log compaction. One weak spot of log compaction is for the
> >> > > consumer
> >> > > > to
> >> > > > > deal with deletes. When a key is deleted, it's retained as a
> >> > tombstone
> >> > > > > first and then physically removed. If a client misses the
> >> tombstone
> >> > > > > (because it's physically removed), it may not be able to update
> >> its
> >> > > > > metadata properly. The way we solve this in Kafka is based on a
> >> > > > > configuration (log.cleaner.delete.retention.ms) and we expect a
> >> > > consumer
> >> > > > > having seen an old key to finish reading the deletion tombstone
> >> > within
> >> > > > that
> >> > > > > time. There is no strong guarantee for that since a broker could
> >> be
> >> > > down
> >> > > > > for a long time. It would be better if we can have a more
> reliable
> >> > way
> >> > > of
> >> > > > > dealing with deletes.
> >> > > > >
> >> > > >
> >> > > > We propose to capture this in the "FirstDirtyOffset" field of the
> >> > quorum
> >> > > > record fetch response: the offset is the maximum offset that log
> >> > > compaction
> >> > > > has reached up to. If the follower has fetched beyond this offset
> it
> >> > > means
> >> > > > itself is safe hence it has seen all records up to that offset. On
> >> > > getting
> >> > > > the response, the follower can then decide if its end offset
> >> actually
> >> > > below
> >> > > > that dirty offset (and hence may miss some tombstones). If that's
> >> the
> >> > > case:
> >> > > >
> >> > > > 1) Naively, it could re-bootstrap metadata log from the very
> >> beginning
> >> > to
> >> > > > catch up.
> >> > > > 2) During that time, it would refrain itself from answering
> >> > > MetadataRequest
> >> > > > from any clients.
> >> > > >
> >> > > >
> >> > > I am not sure that the "FirstDirtyOffset" field fully addresses the
> >> > issue.
> >> > > Currently, the deletion tombstone is not removed immediately after a
> >> > round
> >> > > of cleaning. It's removed after a delay in a subsequent round of
> >> > cleaning.
> >> > > Consider an example where a key insertion is at offset 200 and a
> >> deletion
> >> > > tombstone of the key is at 400. Initially, FirstDirtyOffset is at
> >> 300. A
> >> > > follower/observer fetches from offset 0  and fetches the key at
> offset
> >> > 200.
> >> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
> >> > > tombstone at 400 is physically removed. The follower/observer
> >> continues
> >> > the
> >> > > fetch, but misses offset 400. It catches all the way to
> >> FirstDirtyOffset
> >> > > and declares its metadata as ready. However, its metadata could be
> >> stale
> >> > > since it actually misses the deletion of the key.
> >> > >
> >> > > Yeah good question, I should have put more details in my explanation
> >> :)
> >> >
> >> > The idea is that we will adjust the log compaction for this raft based
> >> > metadata log: before more details to be explained, since we have two
> >> types
> >> > of "watermarks" here, whereas in Kafka the watermark indicates where
> >> every
> >> > replica have replicated up to and in Raft the watermark indicates
> where
> >> the
> >> > majority of replicas (here only indicating voters of the quorum, not
> >> > counting observers) have replicated up to, let's call them Kafka
> >> watermark
> >> > and Raft watermark. For this special log, we would maintain both
> >> > watermarks.
> >> >
> >> > When log compacting on the leader, we would only compact up to the
> Kafka
> >> > watermark, i.e. if there is at least one voter who have not replicated
> >> an
> >> > entry, it would not be compacted. The "dirty-offset" is the offset
> that
> >> > we've compacted up to and is communicated to other voters, and the
> other
> >> > voters would also compact up to this value --- i.e. the difference
> here
> >> is
> >> > that instead of letting each replica doing log compaction
> independently,
> >> > we'll have the leader to decide upon which offset to compact to, and
> >> > propagate this value to others to follow, in a more coordinated
> manner.
> >> > Also note when there are new voters joining the quorum who has not
> >> > replicated up to the dirty-offset, of because of other issues they
> >> > truncated their logs to below the dirty-offset, they'd have to
> >> re-bootstrap
> >> > from the beginning, and during this period of time the leader learned
> >> about
> >> > this lagging voter would not advance the watermark (also it would not
> >> > decrement it), and hence not compacting either, until the voter(s) has
> >> > caught up to that dirty-offset.
> >> >
> >> > So back to your example above, before the bootstrap voter gets to 300
> no
> >> > log compaction would happen on the leader; and until later when the
> >> voter
> >> > have got to beyond 400 and hence replicated that tombstone, the log
> >> > compaction would possibly get to that tombstone and remove it. Say
> >> later it
> >> > the leader's log compaction reaches 500, it can send this back to the
> >> voter
> >> > who can then also compact locally up to 500.
> >> >
> >> >
> >> > > > > 105. Quorum State: In addition to VotedId, do we need the epoch
> >> > > > > corresponding to VotedId? Over time, the same broker Id could be
> >> > voted
> >> > > in
> >> > > > > different generations with different epoch.
> >> > > > >
> >> > > > >
> >> > > > Hmm, this is a good point. Originally I think the "LeaderEpoch"
> >> field
> >> > in
> >> > > > that file is corresponding to the "latest known leader epoch", not
> >> the
> >> > > > "current leader epoch". For example, if the current epoch is N,
> and
> >> > then
> >> > > a
> >> > > > vote-request with epoch N+1 is received and the voter granted the
> >> vote
> >> > > for
> >> > > > it, then it means for this voter it knows the "latest epoch" is N
> +
> >> 1
> >> > > > although it is unknown if that sending candidate will indeed
> become
> >> the
> >> > > new
> >> > > > leader (which would only be notified via begin-quorum request).
> >> > However,
> >> > > > when persisting the quorum state, we would encode leader-epoch to
> >> N+1,
> >> > > > while the leaderId to be the older leader.
> >> > > >
> >> > > > But now thinking about this a bit more, I feel we should use two
> >> > separate
> >> > > > epochs, one for the "lates known" and one for the "current" to
> pair
> >> > with
> >> > > > the leaderId. I will update the wiki page.
> >> > > >
> >> > > >
> >> > > Hmm, it's kind of weird to bump up the leader epoch before the new
> >> leader
> >> > > is actually elected, right.
> >> > >
> >> > >
> >> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to
> >> > > indicate
> >> > > > > that the follower has fetched from an invalid offset and should
> >> > > truncate
> >> > > > to
> >> > > > > the offset/epoch indicated in the response." Observers can't
> >> truncate
> >> > > > their
> >> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> >> > > > >
> >> > > > >
> >> > > > I'm not sure if I understand your question? Observers should still
> >> be
> >> > > able
> >> > > > to truncate their logs as well.
> >> > > >
> >> > > >
> >> > > Hmm, I thought only the quorum nodes have local logs and observers
> >> don't?
> >> > >
> >> > > > 107. "The leader will continue sending BeginQuorumEpoch to each
> >> known
> >> > > > voter
> >> > > > > until it has received its endorsement." If a voter is down for a
> >> long
> >> > > > time,
> >> > > > > sending BeginQuorumEpoch seems to add unnecessary overhead.
> >> > Similarly,
> >> > > > if a
> >> > > > > follower stops sending FetchQuorumRecords, does the leader keep
> >> > sending
> >> > > > > BeginQuorumEpoch?
> >> > > > >
> >> > > >
> >> > > > Regarding BeginQuorumEpoch: that is a good point. The
> >> > begin-quorum-epoch
> >> > > > request is for voters to quickly get the new leader information;
> >> > however
> >> > > > even if they do not get them they can still eventually learn about
> >> that
> >> > > > from others via gossiping FindQuorum. I think we can adjust the
> >> logic
> >> > to
> >> > > > e.g. exponential back-off or with a limited num.retries.
> >> > > >
> >> > > > Regarding FetchQuorumRecords: if the follower sends
> >> FetchQuorumRecords
> >> > > > already, it means that follower already knows that the broker is
> the
> >> > > > leader, and hence we can stop retrying BeginQuorumEpoch; however
> it
> >> is
> >> > > > possible that after a follower sends FetchQuorumRecords already,
> >> > suddenly
> >> > > > it stops send it (possibly because it learned about a higher epoch
> >> > > leader),
> >> > > > and hence this broker may be a "zombie" leader and we propose to
> use
> >> > the
> >> > > > fetch.timeout to let the leader to try to verify if it has already
> >> been
> >> > > > stale.
> >> > > >
> >> > > >
> >> > > It just seems that we should handle these two cases in a consistent
> >> way?
> >> > >
> >> > > Yes I agree, on the leader's side, the FetchQuorumRecords from a
> >> follower
> >> > could mean that we no longer needs to send BeginQuorumEpoch anymore
> ---
> >> and
> >> > it is already part of our current implementations in
> >> > https://github.com/confluentinc/kafka/commits/kafka-raft
> >> >
> >> >
> >> > > Thanks,
> >> > >
> >> > > Jun
> >> > >
> >> > > >
> >> > > > >
> >> > > > > Jun
> >> > > > >
> >> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <
> wangguoz@gmail.com
> >> >
> >> > > > wrote:
> >> > > > >
> >> > > > > > Hello Leonard,
> >> > > > > >
> >> > > > > > Thanks for your comments, I'm relying in line below:
> >> > > > > >
> >> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> >> > wge@confluent.io>
> >> > > > > > wrote:
> >> > > > > >
> >> > > > > > > Hi Kafka developers,
> >> > > > > > >
> >> > > > > > > It's great to see this proposal and it took me some time to
> >> > finish
> >> > > > > > reading
> >> > > > > > > it.
> >> > > > > > >
> >> > > > > > > And I have the following questions about the Proposal:
> >> > > > > > >
> >> > > > > > >    - How do we plan to test this design to ensure its
> >> > correctness?
> >> > > Or
> >> > > > > > more
> >> > > > > > >    broadly, how do we ensure that our new ‘pull’ based model
> >> is
> >> > > > > > functional
> >> > > > > > > and
> >> > > > > > >    correct given that it is different from the original RAFT
> >> > > > > > implementation
> >> > > > > > >    which has formal proof of correctness?
> >> > > > > > >
> >> > > > > >
> >> > > > > > We have two planned verifications on the correctness and
> >> liveness
> >> > of
> >> > > > the
> >> > > > > > design. One is via model verification (TLA+)
> >> > > > > > https://github.com/guozhangwang/kafka-specification
> >> > > > > >
> >> > > > > > Another is via the concurrent simulation tests
> >> > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> >> > > > > >
> >> > > > > >    - Have we considered any sensible defaults for the
> >> > configuration,
> >> > > > i.e.
> >> > > > > > >    all the election timeout, fetch time out, etc.? Or we
> want
> >> to
> >> > > > leave
> >> > > > > > > this to
> >> > > > > > >    a later stage when we do the performance testing, etc.
> >> > > > > > >
> >> > > > > >
> >> > > > > > This is a good question, the reason we did not set any default
> >> > values
> >> > > > for
> >> > > > > > the timeout configurations is that we think it may take some
> >> > > > benchmarking
> >> > > > > > experiments to get these defaults right. Some high-level
> >> principles
> >> > > to
> >> > > > > > consider: 1) the fetch.timeout should be around the same scale
> >> with
> >> > > zk
> >> > > > > > session timeout, which is now 18 seconds by default -- in
> >> practice
> >> > > > we've
> >> > > > > > seen unstable networks having more than 10 secs of transient
> >> > > > > connectivity,
> >> > > > > > 2) the election.timeout, however, should be smaller than the
> >> fetch
> >> > > > > timeout
> >> > > > > > as is also suggested as a practical optimization in
> literature:
> >> > > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> >> > > > > >
> >> > > > > > Some more discussions can be found here:
> >> > > > > >
> https://github.com/confluentinc/kafka/pull/301/files#r415420081
> >> > > > > >
> >> > > > > >
> >> > > > > > >    - Have we considered piggybacking `BeginQuorumEpoch` with
> >> the
> >> > `
> >> > > > > > >    FetchQuorumRecords`? I might be missing something obvious
> >> but
> >> > I
> >> > > am
> >> > > > > > just
> >> > > > > > >    wondering why don’t we just use the `FindQuorum` and
> >> > > > > > > `FetchQuorumRecords`
> >> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> >> > > > > > >
> >> > > > > >
> >> > > > > > Note that Begin/EndQuorumEpoch is sent from leader -> other
> >> voter
> >> > > > > > followers, while FindQuorum / Fetch are sent from follower to
> >> > leader.
> >> > > > > > Arguably one can eventually realize the new leader and epoch
> via
> >> > > > > gossiping
> >> > > > > > FindQuorum, but that could in practice require a long delay.
> >> > Having a
> >> > > > > > leader -> other voters request helps the new leader epoch to
> be
> >> > > > > propagated
> >> > > > > > faster under a pull model.
> >> > > > > >
> >> > > > > >
> >> > > > > > >    - And about the `FetchQuorumRecords` response schema, in
> >> the
> >> > > > > `Records`
> >> > > > > > >    field of the response, is it just one record or all the
> >> > records
> >> > > > > > starting
> >> > > > > > >    from the FetchOffset? It seems a lot more efficient if we
> >> sent
> >> > > all
> >> > > > > the
> >> > > > > > >    records during the bootstrapping of the brokers.
> >> > > > > > >
> >> > > > > >
> >> > > > > > Yes the fetching is batched: FetchOffset is just the starting
> >> > offset
> >> > > of
> >> > > > > the
> >> > > > > > batch of records.
> >> > > > > >
> >> > > > > >
> >> > > > > > >    - Regarding the disruptive broker issues, does our pull
> >> based
> >> > > > model
> >> > > > > > >    suffer from it? If so, have we considered the Pre-Vote
> >> stage?
> >> > If
> >> > > > > not,
> >> > > > > > > why?
> >> > > > > > >
> >> > > > > > >
> >> > > > > > The disruptive broker is stated in the original Raft paper
> >> which is
> >> > > the
> >> > > > > > result of the push model design. Our analysis showed that with
> >> the
> >> > > pull
> >> > > > > > model it is no longer an issue.
> >> > > > > >
> >> > > > > >
> >> > > > > > > Thanks a lot for putting this up, and I hope that my
> questions
> >> > can
> >> > > be
> >> > > > > of
> >> > > > > > > some value to make this KIP better.
> >> > > > > > >
> >> > > > > > > Hope to hear from you soon!
> >> > > > > > >
> >> > > > > > > Best wishes,
> >> > > > > > > Leonard
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> >> cmccabe@apache.org
> >> > >
> >> > > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hi Jason,
> >> > > > > > > >
> >> > > > > > > > It's amazing to see this coming together :)
> >> > > > > > > >
> >> > > > > > > > I haven't had a chance to read in detail, but I read the
> >> > outline
> >> > > > and
> >> > > > > a
> >> > > > > > > few
> >> > > > > > > > things jumped out at me.
> >> > > > > > > >
> >> > > > > > > > First, for every epoch that is 32 bits rather than 64, I
> >> sort
> >> > of
> >> > > > > wonder
> >> > > > > > > if
> >> > > > > > > > that's a good long-term choice.  I keep reading about
> stuff
> >> > like
> >> > > > > this:
> >> > > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
> >> > > Obviously,
> >> > > > > > that
> >> > > > > > > > JIRA is about zxid, which increments much faster than we
> >> expect
> >> > > > these
> >> > > > > > > > leader epochs to, but it would still be good to see some
> >> rough
> >> > > > > > > calculations
> >> > > > > > > > about how long 32 bits (or really, 31 bits) will last us
> in
> >> the
> >> > > > cases
> >> > > > > > > where
> >> > > > > > > > we're using it, and what the space savings we're getting
> >> really
> >> > > is.
> >> > > > > It
> >> > > > > > > > seems like in most cases the tradeoff may not be worth it?
> >> > > > > > > >
> >> > > > > > > > Another thing I've been thinking about is how we do
> >> > > > bootstrapping.  I
> >> > > > > > > > would prefer to be in a world where formatting a new Kafka
> >> node
> >> > > > was a
> >> > > > > > > first
> >> > > > > > > > class operation explicitly initiated by the admin, rather
> >> than
> >> > > > > > something
> >> > > > > > > > that happened implicitly when you started up the broker
> and
> >> > > things
> >> > > > > > > "looked
> >> > > > > > > > blank."
> >> > > > > > > >
> >> > > > > > > > The first problem is that things can "look blank"
> >> accidentally
> >> > if
> >> > > > the
> >> > > > > > > > storage system is having a bad day.  Clearly in the
> non-Raft
> >> > > world,
> >> > > > > > this
> >> > > > > > > > leads to data loss if the broker that is (re)started this
> >> way
> >> > was
> >> > > > the
> >> > > > > > > > leader for some partitions.
> >> > > > > > > >
> >> > > > > > > > The second problem is that we have a bit of a chicken and
> >> egg
> >> > > > problem
> >> > > > > > > with
> >> > > > > > > > certain configuration keys.  For example, maybe you want
> to
> >> > > > configure
> >> > > > > > > some
> >> > > > > > > > connection security settings in your cluster, but you
> don't
> >> > want
> >> > > > them
> >> > > > > > to
> >> > > > > > > > ever be stored in a plaintext config file.  (For example,
> >> SCRAM
> >> > > > > > > passwords,
> >> > > > > > > > etc.)  You could use a broker API to set the
> configuration,
> >> but
> >> > > > that
> >> > > > > > > brings
> >> > > > > > > > up the chicken and egg problem.  The broker needs to be
> >> > > configured
> >> > > > to
> >> > > > > > > know
> >> > > > > > > > how to talk to you, but you need to configure it before
> you
> >> can
> >> > > > talk
> >> > > > > to
> >> > > > > > > > it.  Using an external secret manager like Vault is one
> way
> >> to
> >> > > > solve
> >> > > > > > > this,
> >> > > > > > > > but not everyone uses an external secret manager.
> >> > > > > > > >
> >> > > > > > > > quorum.voters seems like a similar configuration key.  In
> >> the
> >> > > > current
> >> > > > > > > KIP,
> >> > > > > > > > this is only read if there is no other configuration
> >> specifying
> >> > > the
> >> > > > > > > quorum
> >> > > > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't
> need
> >> > this
> >> > > > key
> >> > > > > > > > because we could assume that there was always quorum
> >> > information
> >> > > > > stored
> >> > > > > > > > locally.
> >> > > > > > > >
> >> > > > > > > > best,
> >> > > > > > > > Colin
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> >> > > > > > > > > Hi All,
> >> > > > > > > > >
> >> > > > > > > > > I'd like to start a discussion on KIP-595:
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> >> > > > > > > > .
> >> > > > > > > > > This proposal specifies a Raft protocol to ultimately
> >> replace
> >> > > > > > Zookeeper
> >> > > > > > > > > as
> >> > > > > > > > > documented in KIP-500. Please take a look and share your
> >> > > > thoughts.
> >> > > > > > > > >
> >> > > > > > > > > A few minor notes to set the stage a little bit:
> >> > > > > > > > >
> >> > > > > > > > > - This KIP does not specify the structure of the
> messages
> >> > used
> >> > > to
> >> > > > > > > > represent
> >> > > > > > > > > metadata in Kafka, nor does it specify the internal API
> >> that
> >> > > will
> >> > > > > be
> >> > > > > > > used
> >> > > > > > > > > by the controller. Expect these to come in later
> >> proposals.
> >> > > Here
> >> > > > we
> >> > > > > > are
> >> > > > > > > > > primarily concerned with the replication protocol and
> >> basic
> >> > > > > > operational
> >> > > > > > > > > mechanics.
> >> > > > > > > > > - We expect many details to change as we get closer to
> >> > > > integration
> >> > > > > > with
> >> > > > > > > > > the controller. Any changes we make will be made either
> as
> >> > > > > amendments
> >> > > > > > > to
> >> > > > > > > > > this KIP or, in the case of larger changes, as new
> >> proposals.
> >> > > > > > > > > - We have a prototype implementation which I will put
> >> online
> >> > > > within
> >> > > > > > the
> >> > > > > > > > > next week which may help in understanding some details.
> It
> >> > has
> >> > > > > > > diverged a
> >> > > > > > > > > little bit from our proposal, so I am taking a little
> >> time to
> >> > > > bring
> >> > > > > > it
> >> > > > > > > in
> >> > > > > > > > > line. I'll post an update to this thread when it is
> >> available
> >> > > for
> >> > > > > > > review.
> >> > > > > > > > >
> >> > > > > > > > > Finally, I want to mention that this proposal was
> drafted
> >> by
> >> > > > > myself,
> >> > > > > > > > Boyang
> >> > > > > > > > > Chen, and Guozhang Wang.
> >> > > > > > > > >
> >> > > > > > > > > Thanks,
> >> > > > > > > > > Jason
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > --
> >> > > > > > > Leonard Ge
> >> > > > > > > Software Engineer Intern - Confluent
> >> > > > > > >
> >> > > > > >
> >> > > > > >
> >> > > > > > --
> >> > > > > > -- Guozhang
> >> > > > > >
> >> > > > >
> >> > > >
> >> > > >
> >> > > > --
> >> > > > -- Guozhang
> >> > > >
> >> > >
> >> >
> >> >
> >> > --
> >> > -- Guozhang
> >> >
> >>
> >
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Boyang Chen <re...@gmail.com>.
Hey all,

follow-up on the previous email, we made some more updates:

1. The Alter/DescribeQuorum RPCs are also re-structured to use multi-raft.

2. We add observer status into the DescribeQuorumResponse as we see it is a
low hanging fruit which is very useful for user debugging and reassignment.

3. The FindQuorum RPC is replaced with DiscoverBrokers RPC, which is purely
in charge of discovering broker connections in a gossip manner. The quorum
leader discovery is piggy-back on the Metadata RPC for the topic partition
leader, which in our case is the single metadata partition for the version
one.

Let me know if you have any questions.

Boyang

On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen <re...@gmail.com>
wrote:

> Hey all,
>
> Thanks for the great discussions so far. I'm posting some KIP updates from
> our working group discussion:
>
> 1. We will be changing the core RPCs from single-raft API to multi-raft.
> This means all protocols will be "batch" in the first version, but the KIP
> itself only illustrates the design for a single metadata topic partition.
> The reason is to "keep the door open" for future extensions of this piece
> of module such as a sharded controller or general quorum based topic
> replication, beyond the current Kafka replication protocol.
>
> 2. We will piggy-back on the current Kafka Fetch API instead of inventing
> a new FetchQuorumRecords RPC. The motivation is about the same as #1 as
> well as making the integration work easier, instead of letting two similar
> RPCs diverge.
>
> 3. In the EndQuorumEpoch protocol, instead of only sending the request to
> the most caught-up voter, we shall broadcast the information to all voters,
> with a sorted voter list in descending order of their corresponding
> replicated offset. In this way, the top voter will become a candidate
> immediately, while the other voters shall wait for an exponential back-off
> to trigger elections, which helps ensure the top voter gets elected, and
> the election eventually happens when the top voter is not responsive.
>
> Please see the updated KIP and post any questions or concerns on the
> mailing thread.
>
> Boyang
>
> On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Guozhang and Jason,
>>
>> Thanks for the reply. A couple of more replies.
>>
>> 102. Still not sure about this. How is the tombstone issue addressed in
>> the
>> non-voter and the observer.  They can die at any point and restart at an
>> arbitrary later time, and the advancing of the firstDirty offset and the
>> removal of the tombstone can happen independently.
>>
>> 106. I agree that it would be less confusing if we used "epoch" instead of
>> "leader epoch" consistently.
>>
>> Jun
>>
>> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>> > Thanks Jun. Further replies are in-lined.
>> >
>> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Hi, Guozhang,
>> > >
>> > > Thanks for the reply. A few more replies inlined below.
>> > >
>> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com>
>> wrote:
>> > >
>> > > > Hello Jun,
>> > > >
>> > > > Thanks for your comments! I'm replying inline below:
>> > > >
>> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
>> > > >
>> > > > > 101. Bootstrapping related issues.
>> > > > > 101.1 Currently, we support auto broker id generation. Is this
>> > > supported
>> > > > > for bootstrap brokers?
>> > > > >
>> > > >
>> > > > The vote ids would just be the broker ids. "bootstrap.servers"
>> would be
>> > > > similar to what client configs have today, where "quorum.voters"
>> would
>> > be
>> > > > pre-defined config values.
>> > > >
>> > > >
>> > > My question was on the auto generated broker id. Currently, the broker
>> > can
>> > > choose to have its broker Id auto generated. The generation is done
>> > through
>> > > ZK to guarantee uniqueness. Without ZK, it's not clear how the broker
>> id
>> > is
>> > > auto generated. "quorum.voters" also can't be set statically if broker
>> > ids
>> > > are auto generated.
>> > >
>> > > Jason has explained some ideas that we've discussed so far, the
>> reason we
>> > intentional did not include them so far is that we feel it is out-side
>> the
>> > scope of KIP-595. Under the umbrella of KIP-500 we should definitely
>> > address them though.
>> >
>> > On the high-level, our belief is that "joining a quorum" and "joining
>> (or
>> > more specifically, registering brokers in) the cluster" would be
>> > de-coupled a bit, where the former should be completed before we do the
>> > latter. More specifically, assuming the quorum is already up and
>> running,
>> > after the newly started broker found the leader of the quorum it can
>> send a
>> > specific RegisterBroker request including its listener / protocol / etc,
>> > and upon handling it the leader can send back the uniquely generated
>> broker
>> > id to the new broker, while also executing the "startNewBroker"
>> callback as
>> > the controller.
>> >
>> >
>> > >
>> > > > > 102. Log compaction. One weak spot of log compaction is for the
>> > > consumer
>> > > > to
>> > > > > deal with deletes. When a key is deleted, it's retained as a
>> > tombstone
>> > > > > first and then physically removed. If a client misses the
>> tombstone
>> > > > > (because it's physically removed), it may not be able to update
>> its
>> > > > > metadata properly. The way we solve this in Kafka is based on a
>> > > > > configuration (log.cleaner.delete.retention.ms) and we expect a
>> > > consumer
>> > > > > having seen an old key to finish reading the deletion tombstone
>> > within
>> > > > that
>> > > > > time. There is no strong guarantee for that since a broker could
>> be
>> > > down
>> > > > > for a long time. It would be better if we can have a more reliable
>> > way
>> > > of
>> > > > > dealing with deletes.
>> > > > >
>> > > >
>> > > > We propose to capture this in the "FirstDirtyOffset" field of the
>> > quorum
>> > > > record fetch response: the offset is the maximum offset that log
>> > > compaction
>> > > > has reached up to. If the follower has fetched beyond this offset it
>> > > means
>> > > > itself is safe hence it has seen all records up to that offset. On
>> > > getting
>> > > > the response, the follower can then decide if its end offset
>> actually
>> > > below
>> > > > that dirty offset (and hence may miss some tombstones). If that's
>> the
>> > > case:
>> > > >
>> > > > 1) Naively, it could re-bootstrap metadata log from the very
>> beginning
>> > to
>> > > > catch up.
>> > > > 2) During that time, it would refrain itself from answering
>> > > MetadataRequest
>> > > > from any clients.
>> > > >
>> > > >
>> > > I am not sure that the "FirstDirtyOffset" field fully addresses the
>> > issue.
>> > > Currently, the deletion tombstone is not removed immediately after a
>> > round
>> > > of cleaning. It's removed after a delay in a subsequent round of
>> > cleaning.
>> > > Consider an example where a key insertion is at offset 200 and a
>> deletion
>> > > tombstone of the key is at 400. Initially, FirstDirtyOffset is at
>> 300. A
>> > > follower/observer fetches from offset 0  and fetches the key at offset
>> > 200.
>> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
>> > > tombstone at 400 is physically removed. The follower/observer
>> continues
>> > the
>> > > fetch, but misses offset 400. It catches all the way to
>> FirstDirtyOffset
>> > > and declares its metadata as ready. However, its metadata could be
>> stale
>> > > since it actually misses the deletion of the key.
>> > >
>> > > Yeah good question, I should have put more details in my explanation
>> :)
>> >
>> > The idea is that we will adjust the log compaction for this raft based
>> > metadata log: before more details to be explained, since we have two
>> types
>> > of "watermarks" here, whereas in Kafka the watermark indicates where
>> every
>> > replica have replicated up to and in Raft the watermark indicates where
>> the
>> > majority of replicas (here only indicating voters of the quorum, not
>> > counting observers) have replicated up to, let's call them Kafka
>> watermark
>> > and Raft watermark. For this special log, we would maintain both
>> > watermarks.
>> >
>> > When log compacting on the leader, we would only compact up to the Kafka
>> > watermark, i.e. if there is at least one voter who have not replicated
>> an
>> > entry, it would not be compacted. The "dirty-offset" is the offset that
>> > we've compacted up to and is communicated to other voters, and the other
>> > voters would also compact up to this value --- i.e. the difference here
>> is
>> > that instead of letting each replica doing log compaction independently,
>> > we'll have the leader to decide upon which offset to compact to, and
>> > propagate this value to others to follow, in a more coordinated manner.
>> > Also note when there are new voters joining the quorum who has not
>> > replicated up to the dirty-offset, of because of other issues they
>> > truncated their logs to below the dirty-offset, they'd have to
>> re-bootstrap
>> > from the beginning, and during this period of time the leader learned
>> about
>> > this lagging voter would not advance the watermark (also it would not
>> > decrement it), and hence not compacting either, until the voter(s) has
>> > caught up to that dirty-offset.
>> >
>> > So back to your example above, before the bootstrap voter gets to 300 no
>> > log compaction would happen on the leader; and until later when the
>> voter
>> > have got to beyond 400 and hence replicated that tombstone, the log
>> > compaction would possibly get to that tombstone and remove it. Say
>> later it
>> > the leader's log compaction reaches 500, it can send this back to the
>> voter
>> > who can then also compact locally up to 500.
>> >
>> >
>> > > > > 105. Quorum State: In addition to VotedId, do we need the epoch
>> > > > > corresponding to VotedId? Over time, the same broker Id could be
>> > voted
>> > > in
>> > > > > different generations with different epoch.
>> > > > >
>> > > > >
>> > > > Hmm, this is a good point. Originally I think the "LeaderEpoch"
>> field
>> > in
>> > > > that file is corresponding to the "latest known leader epoch", not
>> the
>> > > > "current leader epoch". For example, if the current epoch is N, and
>> > then
>> > > a
>> > > > vote-request with epoch N+1 is received and the voter granted the
>> vote
>> > > for
>> > > > it, then it means for this voter it knows the "latest epoch" is N +
>> 1
>> > > > although it is unknown if that sending candidate will indeed become
>> the
>> > > new
>> > > > leader (which would only be notified via begin-quorum request).
>> > However,
>> > > > when persisting the quorum state, we would encode leader-epoch to
>> N+1,
>> > > > while the leaderId to be the older leader.
>> > > >
>> > > > But now thinking about this a bit more, I feel we should use two
>> > separate
>> > > > epochs, one for the "lates known" and one for the "current" to pair
>> > with
>> > > > the leaderId. I will update the wiki page.
>> > > >
>> > > >
>> > > Hmm, it's kind of weird to bump up the leader epoch before the new
>> leader
>> > > is actually elected, right.
>> > >
>> > >
>> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to
>> > > indicate
>> > > > > that the follower has fetched from an invalid offset and should
>> > > truncate
>> > > > to
>> > > > > the offset/epoch indicated in the response." Observers can't
>> truncate
>> > > > their
>> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
>> > > > >
>> > > > >
>> > > > I'm not sure if I understand your question? Observers should still
>> be
>> > > able
>> > > > to truncate their logs as well.
>> > > >
>> > > >
>> > > Hmm, I thought only the quorum nodes have local logs and observers
>> don't?
>> > >
>> > > > 107. "The leader will continue sending BeginQuorumEpoch to each
>> known
>> > > > voter
>> > > > > until it has received its endorsement." If a voter is down for a
>> long
>> > > > time,
>> > > > > sending BeginQuorumEpoch seems to add unnecessary overhead.
>> > Similarly,
>> > > > if a
>> > > > > follower stops sending FetchQuorumRecords, does the leader keep
>> > sending
>> > > > > BeginQuorumEpoch?
>> > > > >
>> > > >
>> > > > Regarding BeginQuorumEpoch: that is a good point. The
>> > begin-quorum-epoch
>> > > > request is for voters to quickly get the new leader information;
>> > however
>> > > > even if they do not get them they can still eventually learn about
>> that
>> > > > from others via gossiping FindQuorum. I think we can adjust the
>> logic
>> > to
>> > > > e.g. exponential back-off or with a limited num.retries.
>> > > >
>> > > > Regarding FetchQuorumRecords: if the follower sends
>> FetchQuorumRecords
>> > > > already, it means that follower already knows that the broker is the
>> > > > leader, and hence we can stop retrying BeginQuorumEpoch; however it
>> is
>> > > > possible that after a follower sends FetchQuorumRecords already,
>> > suddenly
>> > > > it stops send it (possibly because it learned about a higher epoch
>> > > leader),
>> > > > and hence this broker may be a "zombie" leader and we propose to use
>> > the
>> > > > fetch.timeout to let the leader to try to verify if it has already
>> been
>> > > > stale.
>> > > >
>> > > >
>> > > It just seems that we should handle these two cases in a consistent
>> way?
>> > >
>> > > Yes I agree, on the leader's side, the FetchQuorumRecords from a
>> follower
>> > could mean that we no longer needs to send BeginQuorumEpoch anymore ---
>> and
>> > it is already part of our current implementations in
>> > https://github.com/confluentinc/kafka/commits/kafka-raft
>> >
>> >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > > >
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wangguoz@gmail.com
>> >
>> > > > wrote:
>> > > > >
>> > > > > > Hello Leonard,
>> > > > > >
>> > > > > > Thanks for your comments, I'm relying in line below:
>> > > > > >
>> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
>> > wge@confluent.io>
>> > > > > > wrote:
>> > > > > >
>> > > > > > > Hi Kafka developers,
>> > > > > > >
>> > > > > > > It's great to see this proposal and it took me some time to
>> > finish
>> > > > > > reading
>> > > > > > > it.
>> > > > > > >
>> > > > > > > And I have the following questions about the Proposal:
>> > > > > > >
>> > > > > > >    - How do we plan to test this design to ensure its
>> > correctness?
>> > > Or
>> > > > > > more
>> > > > > > >    broadly, how do we ensure that our new ‘pull’ based model
>> is
>> > > > > > functional
>> > > > > > > and
>> > > > > > >    correct given that it is different from the original RAFT
>> > > > > > implementation
>> > > > > > >    which has formal proof of correctness?
>> > > > > > >
>> > > > > >
>> > > > > > We have two planned verifications on the correctness and
>> liveness
>> > of
>> > > > the
>> > > > > > design. One is via model verification (TLA+)
>> > > > > > https://github.com/guozhangwang/kafka-specification
>> > > > > >
>> > > > > > Another is via the concurrent simulation tests
>> > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
>> > > > > >
>> > > > > >    - Have we considered any sensible defaults for the
>> > configuration,
>> > > > i.e.
>> > > > > > >    all the election timeout, fetch time out, etc.? Or we want
>> to
>> > > > leave
>> > > > > > > this to
>> > > > > > >    a later stage when we do the performance testing, etc.
>> > > > > > >
>> > > > > >
>> > > > > > This is a good question, the reason we did not set any default
>> > values
>> > > > for
>> > > > > > the timeout configurations is that we think it may take some
>> > > > benchmarking
>> > > > > > experiments to get these defaults right. Some high-level
>> principles
>> > > to
>> > > > > > consider: 1) the fetch.timeout should be around the same scale
>> with
>> > > zk
>> > > > > > session timeout, which is now 18 seconds by default -- in
>> practice
>> > > > we've
>> > > > > > seen unstable networks having more than 10 secs of transient
>> > > > > connectivity,
>> > > > > > 2) the election.timeout, however, should be smaller than the
>> fetch
>> > > > > timeout
>> > > > > > as is also suggested as a practical optimization in literature:
>> > > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
>> > > > > >
>> > > > > > Some more discussions can be found here:
>> > > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
>> > > > > >
>> > > > > >
>> > > > > > >    - Have we considered piggybacking `BeginQuorumEpoch` with
>> the
>> > `
>> > > > > > >    FetchQuorumRecords`? I might be missing something obvious
>> but
>> > I
>> > > am
>> > > > > > just
>> > > > > > >    wondering why don’t we just use the `FindQuorum` and
>> > > > > > > `FetchQuorumRecords`
>> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
>> > > > > > >
>> > > > > >
>> > > > > > Note that Begin/EndQuorumEpoch is sent from leader -> other
>> voter
>> > > > > > followers, while FindQuorum / Fetch are sent from follower to
>> > leader.
>> > > > > > Arguably one can eventually realize the new leader and epoch via
>> > > > > gossiping
>> > > > > > FindQuorum, but that could in practice require a long delay.
>> > Having a
>> > > > > > leader -> other voters request helps the new leader epoch to be
>> > > > > propagated
>> > > > > > faster under a pull model.
>> > > > > >
>> > > > > >
>> > > > > > >    - And about the `FetchQuorumRecords` response schema, in
>> the
>> > > > > `Records`
>> > > > > > >    field of the response, is it just one record or all the
>> > records
>> > > > > > starting
>> > > > > > >    from the FetchOffset? It seems a lot more efficient if we
>> sent
>> > > all
>> > > > > the
>> > > > > > >    records during the bootstrapping of the brokers.
>> > > > > > >
>> > > > > >
>> > > > > > Yes the fetching is batched: FetchOffset is just the starting
>> > offset
>> > > of
>> > > > > the
>> > > > > > batch of records.
>> > > > > >
>> > > > > >
>> > > > > > >    - Regarding the disruptive broker issues, does our pull
>> based
>> > > > model
>> > > > > > >    suffer from it? If so, have we considered the Pre-Vote
>> stage?
>> > If
>> > > > > not,
>> > > > > > > why?
>> > > > > > >
>> > > > > > >
>> > > > > > The disruptive broker is stated in the original Raft paper
>> which is
>> > > the
>> > > > > > result of the push model design. Our analysis showed that with
>> the
>> > > pull
>> > > > > > model it is no longer an issue.
>> > > > > >
>> > > > > >
>> > > > > > > Thanks a lot for putting this up, and I hope that my questions
>> > can
>> > > be
>> > > > > of
>> > > > > > > some value to make this KIP better.
>> > > > > > >
>> > > > > > > Hope to hear from you soon!
>> > > > > > >
>> > > > > > > Best wishes,
>> > > > > > > Leonard
>> > > > > > >
>> > > > > > >
>> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
>> cmccabe@apache.org
>> > >
>> > > > > wrote:
>> > > > > > >
>> > > > > > > > Hi Jason,
>> > > > > > > >
>> > > > > > > > It's amazing to see this coming together :)
>> > > > > > > >
>> > > > > > > > I haven't had a chance to read in detail, but I read the
>> > outline
>> > > > and
>> > > > > a
>> > > > > > > few
>> > > > > > > > things jumped out at me.
>> > > > > > > >
>> > > > > > > > First, for every epoch that is 32 bits rather than 64, I
>> sort
>> > of
>> > > > > wonder
>> > > > > > > if
>> > > > > > > > that's a good long-term choice.  I keep reading about stuff
>> > like
>> > > > > this:
>> > > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
>> > > Obviously,
>> > > > > > that
>> > > > > > > > JIRA is about zxid, which increments much faster than we
>> expect
>> > > > these
>> > > > > > > > leader epochs to, but it would still be good to see some
>> rough
>> > > > > > > calculations
>> > > > > > > > about how long 32 bits (or really, 31 bits) will last us in
>> the
>> > > > cases
>> > > > > > > where
>> > > > > > > > we're using it, and what the space savings we're getting
>> really
>> > > is.
>> > > > > It
>> > > > > > > > seems like in most cases the tradeoff may not be worth it?
>> > > > > > > >
>> > > > > > > > Another thing I've been thinking about is how we do
>> > > > bootstrapping.  I
>> > > > > > > > would prefer to be in a world where formatting a new Kafka
>> node
>> > > > was a
>> > > > > > > first
>> > > > > > > > class operation explicitly initiated by the admin, rather
>> than
>> > > > > > something
>> > > > > > > > that happened implicitly when you started up the broker and
>> > > things
>> > > > > > > "looked
>> > > > > > > > blank."
>> > > > > > > >
>> > > > > > > > The first problem is that things can "look blank"
>> accidentally
>> > if
>> > > > the
>> > > > > > > > storage system is having a bad day.  Clearly in the non-Raft
>> > > world,
>> > > > > > this
>> > > > > > > > leads to data loss if the broker that is (re)started this
>> way
>> > was
>> > > > the
>> > > > > > > > leader for some partitions.
>> > > > > > > >
>> > > > > > > > The second problem is that we have a bit of a chicken and
>> egg
>> > > > problem
>> > > > > > > with
>> > > > > > > > certain configuration keys.  For example, maybe you want to
>> > > > configure
>> > > > > > > some
>> > > > > > > > connection security settings in your cluster, but you don't
>> > want
>> > > > them
>> > > > > > to
>> > > > > > > > ever be stored in a plaintext config file.  (For example,
>> SCRAM
>> > > > > > > passwords,
>> > > > > > > > etc.)  You could use a broker API to set the configuration,
>> but
>> > > > that
>> > > > > > > brings
>> > > > > > > > up the chicken and egg problem.  The broker needs to be
>> > > configured
>> > > > to
>> > > > > > > know
>> > > > > > > > how to talk to you, but you need to configure it before you
>> can
>> > > > talk
>> > > > > to
>> > > > > > > > it.  Using an external secret manager like Vault is one way
>> to
>> > > > solve
>> > > > > > > this,
>> > > > > > > > but not everyone uses an external secret manager.
>> > > > > > > >
>> > > > > > > > quorum.voters seems like a similar configuration key.  In
>> the
>> > > > current
>> > > > > > > KIP,
>> > > > > > > > this is only read if there is no other configuration
>> specifying
>> > > the
>> > > > > > > quorum
>> > > > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't need
>> > this
>> > > > key
>> > > > > > > > because we could assume that there was always quorum
>> > information
>> > > > > stored
>> > > > > > > > locally.
>> > > > > > > >
>> > > > > > > > best,
>> > > > > > > > Colin
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
>> > > > > > > > > Hi All,
>> > > > > > > > >
>> > > > > > > > > I'd like to start a discussion on KIP-595:
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
>> > > > > > > > .
>> > > > > > > > > This proposal specifies a Raft protocol to ultimately
>> replace
>> > > > > > Zookeeper
>> > > > > > > > > as
>> > > > > > > > > documented in KIP-500. Please take a look and share your
>> > > > thoughts.
>> > > > > > > > >
>> > > > > > > > > A few minor notes to set the stage a little bit:
>> > > > > > > > >
>> > > > > > > > > - This KIP does not specify the structure of the messages
>> > used
>> > > to
>> > > > > > > > represent
>> > > > > > > > > metadata in Kafka, nor does it specify the internal API
>> that
>> > > will
>> > > > > be
>> > > > > > > used
>> > > > > > > > > by the controller. Expect these to come in later
>> proposals.
>> > > Here
>> > > > we
>> > > > > > are
>> > > > > > > > > primarily concerned with the replication protocol and
>> basic
>> > > > > > operational
>> > > > > > > > > mechanics.
>> > > > > > > > > - We expect many details to change as we get closer to
>> > > > integration
>> > > > > > with
>> > > > > > > > > the controller. Any changes we make will be made either as
>> > > > > amendments
>> > > > > > > to
>> > > > > > > > > this KIP or, in the case of larger changes, as new
>> proposals.
>> > > > > > > > > - We have a prototype implementation which I will put
>> online
>> > > > within
>> > > > > > the
>> > > > > > > > > next week which may help in understanding some details. It
>> > has
>> > > > > > > diverged a
>> > > > > > > > > little bit from our proposal, so I am taking a little
>> time to
>> > > > bring
>> > > > > > it
>> > > > > > > in
>> > > > > > > > > line. I'll post an update to this thread when it is
>> available
>> > > for
>> > > > > > > review.
>> > > > > > > > >
>> > > > > > > > > Finally, I want to mention that this proposal was drafted
>> by
>> > > > > myself,
>> > > > > > > > Boyang
>> > > > > > > > > Chen, and Guozhang Wang.
>> > > > > > > > >
>> > > > > > > > > Thanks,
>> > > > > > > > > Jason
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > --
>> > > > > > > Leonard Ge
>> > > > > > > Software Engineer Intern - Confluent
>> > > > > > >
>> > > > > >
>> > > > > >
>> > > > > > --
>> > > > > > -- Guozhang
>> > > > > >
>> > > > >
>> > > >
>> > > >
>> > > > --
>> > > > -- Guozhang
>> > > >
>> > >
>> >
>> >
>> > --
>> > -- Guozhang
>> >
>>
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Boyang Chen <re...@gmail.com>.
Hey all,

Thanks for the great discussions so far. I'm posting some KIP updates from
our working group discussion:

1. We will be changing the core RPCs from single-raft API to multi-raft.
This means all protocols will be "batch" in the first version, but the KIP
itself only illustrates the design for a single metadata topic partition.
The reason is to "keep the door open" for future extensions of this piece
of module such as a sharded controller or general quorum based topic
replication, beyond the current Kafka replication protocol.

2. We will piggy-back on the current Kafka Fetch API instead of inventing a
new FetchQuorumRecords RPC. The motivation is about the same as #1 as well
as making the integration work easier, instead of letting two similar RPCs
diverge.

3. In the EndQuorumEpoch protocol, instead of only sending the request to
the most caught-up voter, we shall broadcast the information to all voters,
with a sorted voter list in descending order of their corresponding
replicated offset. In this way, the top voter will become a candidate
immediately, while the other voters shall wait for an exponential back-off
to trigger elections, which helps ensure the top voter gets elected, and
the election eventually happens when the top voter is not responsive.

Please see the updated KIP and post any questions or concerns on the
mailing thread.

Boyang

On Fri, May 8, 2020 at 5:26 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Guozhang and Jason,
>
> Thanks for the reply. A couple of more replies.
>
> 102. Still not sure about this. How is the tombstone issue addressed in the
> non-voter and the observer.  They can die at any point and restart at an
> arbitrary later time, and the advancing of the firstDirty offset and the
> removal of the tombstone can happen independently.
>
> 106. I agree that it would be less confusing if we used "epoch" instead of
> "leader epoch" consistently.
>
> Jun
>
> On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Thanks Jun. Further replies are in-lined.
> >
> > On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Guozhang,
> > >
> > > Thanks for the reply. A few more replies inlined below.
> > >
> > > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> > >
> > > > Hello Jun,
> > > >
> > > > Thanks for your comments! I'm replying inline below:
> > > >
> > > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > 101. Bootstrapping related issues.
> > > > > 101.1 Currently, we support auto broker id generation. Is this
> > > supported
> > > > > for bootstrap brokers?
> > > > >
> > > >
> > > > The vote ids would just be the broker ids. "bootstrap.servers" would
> be
> > > > similar to what client configs have today, where "quorum.voters"
> would
> > be
> > > > pre-defined config values.
> > > >
> > > >
> > > My question was on the auto generated broker id. Currently, the broker
> > can
> > > choose to have its broker Id auto generated. The generation is done
> > through
> > > ZK to guarantee uniqueness. Without ZK, it's not clear how the broker
> id
> > is
> > > auto generated. "quorum.voters" also can't be set statically if broker
> > ids
> > > are auto generated.
> > >
> > > Jason has explained some ideas that we've discussed so far, the reason
> we
> > intentional did not include them so far is that we feel it is out-side
> the
> > scope of KIP-595. Under the umbrella of KIP-500 we should definitely
> > address them though.
> >
> > On the high-level, our belief is that "joining a quorum" and "joining (or
> > more specifically, registering brokers in) the cluster" would be
> > de-coupled a bit, where the former should be completed before we do the
> > latter. More specifically, assuming the quorum is already up and running,
> > after the newly started broker found the leader of the quorum it can
> send a
> > specific RegisterBroker request including its listener / protocol / etc,
> > and upon handling it the leader can send back the uniquely generated
> broker
> > id to the new broker, while also executing the "startNewBroker" callback
> as
> > the controller.
> >
> >
> > >
> > > > > 102. Log compaction. One weak spot of log compaction is for the
> > > consumer
> > > > to
> > > > > deal with deletes. When a key is deleted, it's retained as a
> > tombstone
> > > > > first and then physically removed. If a client misses the tombstone
> > > > > (because it's physically removed), it may not be able to update its
> > > > > metadata properly. The way we solve this in Kafka is based on a
> > > > > configuration (log.cleaner.delete.retention.ms) and we expect a
> > > consumer
> > > > > having seen an old key to finish reading the deletion tombstone
> > within
> > > > that
> > > > > time. There is no strong guarantee for that since a broker could be
> > > down
> > > > > for a long time. It would be better if we can have a more reliable
> > way
> > > of
> > > > > dealing with deletes.
> > > > >
> > > >
> > > > We propose to capture this in the "FirstDirtyOffset" field of the
> > quorum
> > > > record fetch response: the offset is the maximum offset that log
> > > compaction
> > > > has reached up to. If the follower has fetched beyond this offset it
> > > means
> > > > itself is safe hence it has seen all records up to that offset. On
> > > getting
> > > > the response, the follower can then decide if its end offset actually
> > > below
> > > > that dirty offset (and hence may miss some tombstones). If that's the
> > > case:
> > > >
> > > > 1) Naively, it could re-bootstrap metadata log from the very
> beginning
> > to
> > > > catch up.
> > > > 2) During that time, it would refrain itself from answering
> > > MetadataRequest
> > > > from any clients.
> > > >
> > > >
> > > I am not sure that the "FirstDirtyOffset" field fully addresses the
> > issue.
> > > Currently, the deletion tombstone is not removed immediately after a
> > round
> > > of cleaning. It's removed after a delay in a subsequent round of
> > cleaning.
> > > Consider an example where a key insertion is at offset 200 and a
> deletion
> > > tombstone of the key is at 400. Initially, FirstDirtyOffset is at 300.
> A
> > > follower/observer fetches from offset 0  and fetches the key at offset
> > 200.
> > > A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
> > > tombstone at 400 is physically removed. The follower/observer continues
> > the
> > > fetch, but misses offset 400. It catches all the way to
> FirstDirtyOffset
> > > and declares its metadata as ready. However, its metadata could be
> stale
> > > since it actually misses the deletion of the key.
> > >
> > > Yeah good question, I should have put more details in my explanation :)
> >
> > The idea is that we will adjust the log compaction for this raft based
> > metadata log: before more details to be explained, since we have two
> types
> > of "watermarks" here, whereas in Kafka the watermark indicates where
> every
> > replica have replicated up to and in Raft the watermark indicates where
> the
> > majority of replicas (here only indicating voters of the quorum, not
> > counting observers) have replicated up to, let's call them Kafka
> watermark
> > and Raft watermark. For this special log, we would maintain both
> > watermarks.
> >
> > When log compacting on the leader, we would only compact up to the Kafka
> > watermark, i.e. if there is at least one voter who have not replicated an
> > entry, it would not be compacted. The "dirty-offset" is the offset that
> > we've compacted up to and is communicated to other voters, and the other
> > voters would also compact up to this value --- i.e. the difference here
> is
> > that instead of letting each replica doing log compaction independently,
> > we'll have the leader to decide upon which offset to compact to, and
> > propagate this value to others to follow, in a more coordinated manner.
> > Also note when there are new voters joining the quorum who has not
> > replicated up to the dirty-offset, of because of other issues they
> > truncated their logs to below the dirty-offset, they'd have to
> re-bootstrap
> > from the beginning, and during this period of time the leader learned
> about
> > this lagging voter would not advance the watermark (also it would not
> > decrement it), and hence not compacting either, until the voter(s) has
> > caught up to that dirty-offset.
> >
> > So back to your example above, before the bootstrap voter gets to 300 no
> > log compaction would happen on the leader; and until later when the voter
> > have got to beyond 400 and hence replicated that tombstone, the log
> > compaction would possibly get to that tombstone and remove it. Say later
> it
> > the leader's log compaction reaches 500, it can send this back to the
> voter
> > who can then also compact locally up to 500.
> >
> >
> > > > > 105. Quorum State: In addition to VotedId, do we need the epoch
> > > > > corresponding to VotedId? Over time, the same broker Id could be
> > voted
> > > in
> > > > > different generations with different epoch.
> > > > >
> > > > >
> > > > Hmm, this is a good point. Originally I think the "LeaderEpoch" field
> > in
> > > > that file is corresponding to the "latest known leader epoch", not
> the
> > > > "current leader epoch". For example, if the current epoch is N, and
> > then
> > > a
> > > > vote-request with epoch N+1 is received and the voter granted the
> vote
> > > for
> > > > it, then it means for this voter it knows the "latest epoch" is N + 1
> > > > although it is unknown if that sending candidate will indeed become
> the
> > > new
> > > > leader (which would only be notified via begin-quorum request).
> > However,
> > > > when persisting the quorum state, we would encode leader-epoch to
> N+1,
> > > > while the leaderId to be the older leader.
> > > >
> > > > But now thinking about this a bit more, I feel we should use two
> > separate
> > > > epochs, one for the "lates known" and one for the "current" to pair
> > with
> > > > the leaderId. I will update the wiki page.
> > > >
> > > >
> > > Hmm, it's kind of weird to bump up the leader epoch before the new
> leader
> > > is actually elected, right.
> > >
> > >
> > > > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to
> > > indicate
> > > > > that the follower has fetched from an invalid offset and should
> > > truncate
> > > > to
> > > > > the offset/epoch indicated in the response." Observers can't
> truncate
> > > > their
> > > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > > >
> > > > >
> > > > I'm not sure if I understand your question? Observers should still be
> > > able
> > > > to truncate their logs as well.
> > > >
> > > >
> > > Hmm, I thought only the quorum nodes have local logs and observers
> don't?
> > >
> > > > 107. "The leader will continue sending BeginQuorumEpoch to each known
> > > > voter
> > > > > until it has received its endorsement." If a voter is down for a
> long
> > > > time,
> > > > > sending BeginQuorumEpoch seems to add unnecessary overhead.
> > Similarly,
> > > > if a
> > > > > follower stops sending FetchQuorumRecords, does the leader keep
> > sending
> > > > > BeginQuorumEpoch?
> > > > >
> > > >
> > > > Regarding BeginQuorumEpoch: that is a good point. The
> > begin-quorum-epoch
> > > > request is for voters to quickly get the new leader information;
> > however
> > > > even if they do not get them they can still eventually learn about
> that
> > > > from others via gossiping FindQuorum. I think we can adjust the logic
> > to
> > > > e.g. exponential back-off or with a limited num.retries.
> > > >
> > > > Regarding FetchQuorumRecords: if the follower sends
> FetchQuorumRecords
> > > > already, it means that follower already knows that the broker is the
> > > > leader, and hence we can stop retrying BeginQuorumEpoch; however it
> is
> > > > possible that after a follower sends FetchQuorumRecords already,
> > suddenly
> > > > it stops send it (possibly because it learned about a higher epoch
> > > leader),
> > > > and hence this broker may be a "zombie" leader and we propose to use
> > the
> > > > fetch.timeout to let the leader to try to verify if it has already
> been
> > > > stale.
> > > >
> > > >
> > > It just seems that we should handle these two cases in a consistent
> way?
> > >
> > > Yes I agree, on the leader's side, the FetchQuorumRecords from a
> follower
> > could mean that we no longer needs to send BeginQuorumEpoch anymore ---
> and
> > it is already part of our current implementations in
> > https://github.com/confluentinc/kafka/commits/kafka-raft
> >
> >
> > > Thanks,
> > >
> > > Jun
> > >
> > > >
> > > > >
> > > > > Jun
> > > > >
> > > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hello Leonard,
> > > > > >
> > > > > > Thanks for your comments, I'm relying in line below:
> > > > > >
> > > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> > wge@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Kafka developers,
> > > > > > >
> > > > > > > It's great to see this proposal and it took me some time to
> > finish
> > > > > > reading
> > > > > > > it.
> > > > > > >
> > > > > > > And I have the following questions about the Proposal:
> > > > > > >
> > > > > > >    - How do we plan to test this design to ensure its
> > correctness?
> > > Or
> > > > > > more
> > > > > > >    broadly, how do we ensure that our new ‘pull’ based model is
> > > > > > functional
> > > > > > > and
> > > > > > >    correct given that it is different from the original RAFT
> > > > > > implementation
> > > > > > >    which has formal proof of correctness?
> > > > > > >
> > > > > >
> > > > > > We have two planned verifications on the correctness and liveness
> > of
> > > > the
> > > > > > design. One is via model verification (TLA+)
> > > > > > https://github.com/guozhangwang/kafka-specification
> > > > > >
> > > > > > Another is via the concurrent simulation tests
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > > >
> > > > > >    - Have we considered any sensible defaults for the
> > configuration,
> > > > i.e.
> > > > > > >    all the election timeout, fetch time out, etc.? Or we want
> to
> > > > leave
> > > > > > > this to
> > > > > > >    a later stage when we do the performance testing, etc.
> > > > > > >
> > > > > >
> > > > > > This is a good question, the reason we did not set any default
> > values
> > > > for
> > > > > > the timeout configurations is that we think it may take some
> > > > benchmarking
> > > > > > experiments to get these defaults right. Some high-level
> principles
> > > to
> > > > > > consider: 1) the fetch.timeout should be around the same scale
> with
> > > zk
> > > > > > session timeout, which is now 18 seconds by default -- in
> practice
> > > > we've
> > > > > > seen unstable networks having more than 10 secs of transient
> > > > > connectivity,
> > > > > > 2) the election.timeout, however, should be smaller than the
> fetch
> > > > > timeout
> > > > > > as is also suggested as a practical optimization in literature:
> > > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > > >
> > > > > > Some more discussions can be found here:
> > > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > > >
> > > > > >
> > > > > > >    - Have we considered piggybacking `BeginQuorumEpoch` with
> the
> > `
> > > > > > >    FetchQuorumRecords`? I might be missing something obvious
> but
> > I
> > > am
> > > > > > just
> > > > > > >    wondering why don’t we just use the `FindQuorum` and
> > > > > > > `FetchQuorumRecords`
> > > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > > >
> > > > > >
> > > > > > Note that Begin/EndQuorumEpoch is sent from leader -> other voter
> > > > > > followers, while FindQuorum / Fetch are sent from follower to
> > leader.
> > > > > > Arguably one can eventually realize the new leader and epoch via
> > > > > gossiping
> > > > > > FindQuorum, but that could in practice require a long delay.
> > Having a
> > > > > > leader -> other voters request helps the new leader epoch to be
> > > > > propagated
> > > > > > faster under a pull model.
> > > > > >
> > > > > >
> > > > > > >    - And about the `FetchQuorumRecords` response schema, in the
> > > > > `Records`
> > > > > > >    field of the response, is it just one record or all the
> > records
> > > > > > starting
> > > > > > >    from the FetchOffset? It seems a lot more efficient if we
> sent
> > > all
> > > > > the
> > > > > > >    records during the bootstrapping of the brokers.
> > > > > > >
> > > > > >
> > > > > > Yes the fetching is batched: FetchOffset is just the starting
> > offset
> > > of
> > > > > the
> > > > > > batch of records.
> > > > > >
> > > > > >
> > > > > > >    - Regarding the disruptive broker issues, does our pull
> based
> > > > model
> > > > > > >    suffer from it? If so, have we considered the Pre-Vote
> stage?
> > If
> > > > > not,
> > > > > > > why?
> > > > > > >
> > > > > > >
> > > > > > The disruptive broker is stated in the original Raft paper which
> is
> > > the
> > > > > > result of the push model design. Our analysis showed that with
> the
> > > pull
> > > > > > model it is no longer an issue.
> > > > > >
> > > > > >
> > > > > > > Thanks a lot for putting this up, and I hope that my questions
> > can
> > > be
> > > > > of
> > > > > > > some value to make this KIP better.
> > > > > > >
> > > > > > > Hope to hear from you soon!
> > > > > > >
> > > > > > > Best wishes,
> > > > > > > Leonard
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <
> cmccabe@apache.org
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi Jason,
> > > > > > > >
> > > > > > > > It's amazing to see this coming together :)
> > > > > > > >
> > > > > > > > I haven't had a chance to read in detail, but I read the
> > outline
> > > > and
> > > > > a
> > > > > > > few
> > > > > > > > things jumped out at me.
> > > > > > > >
> > > > > > > > First, for every epoch that is 32 bits rather than 64, I sort
> > of
> > > > > wonder
> > > > > > > if
> > > > > > > > that's a good long-term choice.  I keep reading about stuff
> > like
> > > > > this:
> > > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
> > > Obviously,
> > > > > > that
> > > > > > > > JIRA is about zxid, which increments much faster than we
> expect
> > > > these
> > > > > > > > leader epochs to, but it would still be good to see some
> rough
> > > > > > > calculations
> > > > > > > > about how long 32 bits (or really, 31 bits) will last us in
> the
> > > > cases
> > > > > > > where
> > > > > > > > we're using it, and what the space savings we're getting
> really
> > > is.
> > > > > It
> > > > > > > > seems like in most cases the tradeoff may not be worth it?
> > > > > > > >
> > > > > > > > Another thing I've been thinking about is how we do
> > > > bootstrapping.  I
> > > > > > > > would prefer to be in a world where formatting a new Kafka
> node
> > > > was a
> > > > > > > first
> > > > > > > > class operation explicitly initiated by the admin, rather
> than
> > > > > > something
> > > > > > > > that happened implicitly when you started up the broker and
> > > things
> > > > > > > "looked
> > > > > > > > blank."
> > > > > > > >
> > > > > > > > The first problem is that things can "look blank"
> accidentally
> > if
> > > > the
> > > > > > > > storage system is having a bad day.  Clearly in the non-Raft
> > > world,
> > > > > > this
> > > > > > > > leads to data loss if the broker that is (re)started this way
> > was
> > > > the
> > > > > > > > leader for some partitions.
> > > > > > > >
> > > > > > > > The second problem is that we have a bit of a chicken and egg
> > > > problem
> > > > > > > with
> > > > > > > > certain configuration keys.  For example, maybe you want to
> > > > configure
> > > > > > > some
> > > > > > > > connection security settings in your cluster, but you don't
> > want
> > > > them
> > > > > > to
> > > > > > > > ever be stored in a plaintext config file.  (For example,
> SCRAM
> > > > > > > passwords,
> > > > > > > > etc.)  You could use a broker API to set the configuration,
> but
> > > > that
> > > > > > > brings
> > > > > > > > up the chicken and egg problem.  The broker needs to be
> > > configured
> > > > to
> > > > > > > know
> > > > > > > > how to talk to you, but you need to configure it before you
> can
> > > > talk
> > > > > to
> > > > > > > > it.  Using an external secret manager like Vault is one way
> to
> > > > solve
> > > > > > > this,
> > > > > > > > but not everyone uses an external secret manager.
> > > > > > > >
> > > > > > > > quorum.voters seems like a similar configuration key.  In the
> > > > current
> > > > > > > KIP,
> > > > > > > > this is only read if there is no other configuration
> specifying
> > > the
> > > > > > > quorum
> > > > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't need
> > this
> > > > key
> > > > > > > > because we could assume that there was always quorum
> > information
> > > > > stored
> > > > > > > > locally.
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > > > > > > > Hi All,
> > > > > > > > >
> > > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > > > .
> > > > > > > > > This proposal specifies a Raft protocol to ultimately
> replace
> > > > > > Zookeeper
> > > > > > > > > as
> > > > > > > > > documented in KIP-500. Please take a look and share your
> > > > thoughts.
> > > > > > > > >
> > > > > > > > > A few minor notes to set the stage a little bit:
> > > > > > > > >
> > > > > > > > > - This KIP does not specify the structure of the messages
> > used
> > > to
> > > > > > > > represent
> > > > > > > > > metadata in Kafka, nor does it specify the internal API
> that
> > > will
> > > > > be
> > > > > > > used
> > > > > > > > > by the controller. Expect these to come in later proposals.
> > > Here
> > > > we
> > > > > > are
> > > > > > > > > primarily concerned with the replication protocol and basic
> > > > > > operational
> > > > > > > > > mechanics.
> > > > > > > > > - We expect many details to change as we get closer to
> > > > integration
> > > > > > with
> > > > > > > > > the controller. Any changes we make will be made either as
> > > > > amendments
> > > > > > > to
> > > > > > > > > this KIP or, in the case of larger changes, as new
> proposals.
> > > > > > > > > - We have a prototype implementation which I will put
> online
> > > > within
> > > > > > the
> > > > > > > > > next week which may help in understanding some details. It
> > has
> > > > > > > diverged a
> > > > > > > > > little bit from our proposal, so I am taking a little time
> to
> > > > bring
> > > > > > it
> > > > > > > in
> > > > > > > > > line. I'll post an update to this thread when it is
> available
> > > for
> > > > > > > review.
> > > > > > > > >
> > > > > > > > > Finally, I want to mention that this proposal was drafted
> by
> > > > > myself,
> > > > > > > > Boyang
> > > > > > > > > Chen, and Guozhang Wang.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Jason
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > Leonard Ge
> > > > > > > Software Engineer Intern - Confluent
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jun Rao <ju...@confluent.io>.
Hi, Guozhang and Jason,

Thanks for the reply. A couple of more replies.

102. Still not sure about this. How is the tombstone issue addressed in the
non-voter and the observer.  They can die at any point and restart at an
arbitrary later time, and the advancing of the firstDirty offset and the
removal of the tombstone can happen independently.

106. I agree that it would be less confusing if we used "epoch" instead of
"leader epoch" consistently.

Jun

On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com> wrote:

> Thanks Jun. Further replies are in-lined.
>
> On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Guozhang,
> >
> > Thanks for the reply. A few more replies inlined below.
> >
> > On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com> wrote:
> >
> > > Hello Jun,
> > >
> > > Thanks for your comments! I'm replying inline below:
> > >
> > > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > 101. Bootstrapping related issues.
> > > > 101.1 Currently, we support auto broker id generation. Is this
> > supported
> > > > for bootstrap brokers?
> > > >
> > >
> > > The vote ids would just be the broker ids. "bootstrap.servers" would be
> > > similar to what client configs have today, where "quorum.voters" would
> be
> > > pre-defined config values.
> > >
> > >
> > My question was on the auto generated broker id. Currently, the broker
> can
> > choose to have its broker Id auto generated. The generation is done
> through
> > ZK to guarantee uniqueness. Without ZK, it's not clear how the broker id
> is
> > auto generated. "quorum.voters" also can't be set statically if broker
> ids
> > are auto generated.
> >
> > Jason has explained some ideas that we've discussed so far, the reason we
> intentional did not include them so far is that we feel it is out-side the
> scope of KIP-595. Under the umbrella of KIP-500 we should definitely
> address them though.
>
> On the high-level, our belief is that "joining a quorum" and "joining (or
> more specifically, registering brokers in) the cluster" would be
> de-coupled a bit, where the former should be completed before we do the
> latter. More specifically, assuming the quorum is already up and running,
> after the newly started broker found the leader of the quorum it can send a
> specific RegisterBroker request including its listener / protocol / etc,
> and upon handling it the leader can send back the uniquely generated broker
> id to the new broker, while also executing the "startNewBroker" callback as
> the controller.
>
>
> >
> > > > 102. Log compaction. One weak spot of log compaction is for the
> > consumer
> > > to
> > > > deal with deletes. When a key is deleted, it's retained as a
> tombstone
> > > > first and then physically removed. If a client misses the tombstone
> > > > (because it's physically removed), it may not be able to update its
> > > > metadata properly. The way we solve this in Kafka is based on a
> > > > configuration (log.cleaner.delete.retention.ms) and we expect a
> > consumer
> > > > having seen an old key to finish reading the deletion tombstone
> within
> > > that
> > > > time. There is no strong guarantee for that since a broker could be
> > down
> > > > for a long time. It would be better if we can have a more reliable
> way
> > of
> > > > dealing with deletes.
> > > >
> > >
> > > We propose to capture this in the "FirstDirtyOffset" field of the
> quorum
> > > record fetch response: the offset is the maximum offset that log
> > compaction
> > > has reached up to. If the follower has fetched beyond this offset it
> > means
> > > itself is safe hence it has seen all records up to that offset. On
> > getting
> > > the response, the follower can then decide if its end offset actually
> > below
> > > that dirty offset (and hence may miss some tombstones). If that's the
> > case:
> > >
> > > 1) Naively, it could re-bootstrap metadata log from the very beginning
> to
> > > catch up.
> > > 2) During that time, it would refrain itself from answering
> > MetadataRequest
> > > from any clients.
> > >
> > >
> > I am not sure that the "FirstDirtyOffset" field fully addresses the
> issue.
> > Currently, the deletion tombstone is not removed immediately after a
> round
> > of cleaning. It's removed after a delay in a subsequent round of
> cleaning.
> > Consider an example where a key insertion is at offset 200 and a deletion
> > tombstone of the key is at 400. Initially, FirstDirtyOffset is at 300. A
> > follower/observer fetches from offset 0  and fetches the key at offset
> 200.
> > A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
> > tombstone at 400 is physically removed. The follower/observer continues
> the
> > fetch, but misses offset 400. It catches all the way to FirstDirtyOffset
> > and declares its metadata as ready. However, its metadata could be stale
> > since it actually misses the deletion of the key.
> >
> > Yeah good question, I should have put more details in my explanation :)
>
> The idea is that we will adjust the log compaction for this raft based
> metadata log: before more details to be explained, since we have two types
> of "watermarks" here, whereas in Kafka the watermark indicates where every
> replica have replicated up to and in Raft the watermark indicates where the
> majority of replicas (here only indicating voters of the quorum, not
> counting observers) have replicated up to, let's call them Kafka watermark
> and Raft watermark. For this special log, we would maintain both
> watermarks.
>
> When log compacting on the leader, we would only compact up to the Kafka
> watermark, i.e. if there is at least one voter who have not replicated an
> entry, it would not be compacted. The "dirty-offset" is the offset that
> we've compacted up to and is communicated to other voters, and the other
> voters would also compact up to this value --- i.e. the difference here is
> that instead of letting each replica doing log compaction independently,
> we'll have the leader to decide upon which offset to compact to, and
> propagate this value to others to follow, in a more coordinated manner.
> Also note when there are new voters joining the quorum who has not
> replicated up to the dirty-offset, of because of other issues they
> truncated their logs to below the dirty-offset, they'd have to re-bootstrap
> from the beginning, and during this period of time the leader learned about
> this lagging voter would not advance the watermark (also it would not
> decrement it), and hence not compacting either, until the voter(s) has
> caught up to that dirty-offset.
>
> So back to your example above, before the bootstrap voter gets to 300 no
> log compaction would happen on the leader; and until later when the voter
> have got to beyond 400 and hence replicated that tombstone, the log
> compaction would possibly get to that tombstone and remove it. Say later it
> the leader's log compaction reaches 500, it can send this back to the voter
> who can then also compact locally up to 500.
>
>
> > > > 105. Quorum State: In addition to VotedId, do we need the epoch
> > > > corresponding to VotedId? Over time, the same broker Id could be
> voted
> > in
> > > > different generations with different epoch.
> > > >
> > > >
> > > Hmm, this is a good point. Originally I think the "LeaderEpoch" field
> in
> > > that file is corresponding to the "latest known leader epoch", not the
> > > "current leader epoch". For example, if the current epoch is N, and
> then
> > a
> > > vote-request with epoch N+1 is received and the voter granted the vote
> > for
> > > it, then it means for this voter it knows the "latest epoch" is N + 1
> > > although it is unknown if that sending candidate will indeed become the
> > new
> > > leader (which would only be notified via begin-quorum request).
> However,
> > > when persisting the quorum state, we would encode leader-epoch to N+1,
> > > while the leaderId to be the older leader.
> > >
> > > But now thinking about this a bit more, I feel we should use two
> separate
> > > epochs, one for the "lates known" and one for the "current" to pair
> with
> > > the leaderId. I will update the wiki page.
> > >
> > >
> > Hmm, it's kind of weird to bump up the leader epoch before the new leader
> > is actually elected, right.
> >
> >
> > > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to
> > indicate
> > > > that the follower has fetched from an invalid offset and should
> > truncate
> > > to
> > > > the offset/epoch indicated in the response." Observers can't truncate
> > > their
> > > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > > >
> > > >
> > > I'm not sure if I understand your question? Observers should still be
> > able
> > > to truncate their logs as well.
> > >
> > >
> > Hmm, I thought only the quorum nodes have local logs and observers don't?
> >
> > > 107. "The leader will continue sending BeginQuorumEpoch to each known
> > > voter
> > > > until it has received its endorsement." If a voter is down for a long
> > > time,
> > > > sending BeginQuorumEpoch seems to add unnecessary overhead.
> Similarly,
> > > if a
> > > > follower stops sending FetchQuorumRecords, does the leader keep
> sending
> > > > BeginQuorumEpoch?
> > > >
> > >
> > > Regarding BeginQuorumEpoch: that is a good point. The
> begin-quorum-epoch
> > > request is for voters to quickly get the new leader information;
> however
> > > even if they do not get them they can still eventually learn about that
> > > from others via gossiping FindQuorum. I think we can adjust the logic
> to
> > > e.g. exponential back-off or with a limited num.retries.
> > >
> > > Regarding FetchQuorumRecords: if the follower sends FetchQuorumRecords
> > > already, it means that follower already knows that the broker is the
> > > leader, and hence we can stop retrying BeginQuorumEpoch; however it is
> > > possible that after a follower sends FetchQuorumRecords already,
> suddenly
> > > it stops send it (possibly because it learned about a higher epoch
> > leader),
> > > and hence this broker may be a "zombie" leader and we propose to use
> the
> > > fetch.timeout to let the leader to try to verify if it has already been
> > > stale.
> > >
> > >
> > It just seems that we should handle these two cases in a consistent way?
> >
> > Yes I agree, on the leader's side, the FetchQuorumRecords from a follower
> could mean that we no longer needs to send BeginQuorumEpoch anymore --- and
> it is already part of our current implementations in
> https://github.com/confluentinc/kafka/commits/kafka-raft
>
>
> > Thanks,
> >
> > Jun
> >
> > >
> > > >
> > > > Jun
> > > >
> > > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hello Leonard,
> > > > >
> > > > > Thanks for your comments, I'm relying in line below:
> > > > >
> > > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <
> wge@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Hi Kafka developers,
> > > > > >
> > > > > > It's great to see this proposal and it took me some time to
> finish
> > > > > reading
> > > > > > it.
> > > > > >
> > > > > > And I have the following questions about the Proposal:
> > > > > >
> > > > > >    - How do we plan to test this design to ensure its
> correctness?
> > Or
> > > > > more
> > > > > >    broadly, how do we ensure that our new ‘pull’ based model is
> > > > > functional
> > > > > > and
> > > > > >    correct given that it is different from the original RAFT
> > > > > implementation
> > > > > >    which has formal proof of correctness?
> > > > > >
> > > > >
> > > > > We have two planned verifications on the correctness and liveness
> of
> > > the
> > > > > design. One is via model verification (TLA+)
> > > > > https://github.com/guozhangwang/kafka-specification
> > > > >
> > > > > Another is via the concurrent simulation tests
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > > >
> > > > >    - Have we considered any sensible defaults for the
> configuration,
> > > i.e.
> > > > > >    all the election timeout, fetch time out, etc.? Or we want to
> > > leave
> > > > > > this to
> > > > > >    a later stage when we do the performance testing, etc.
> > > > > >
> > > > >
> > > > > This is a good question, the reason we did not set any default
> values
> > > for
> > > > > the timeout configurations is that we think it may take some
> > > benchmarking
> > > > > experiments to get these defaults right. Some high-level principles
> > to
> > > > > consider: 1) the fetch.timeout should be around the same scale with
> > zk
> > > > > session timeout, which is now 18 seconds by default -- in practice
> > > we've
> > > > > seen unstable networks having more than 10 secs of transient
> > > > connectivity,
> > > > > 2) the election.timeout, however, should be smaller than the fetch
> > > > timeout
> > > > > as is also suggested as a practical optimization in literature:
> > > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > > >
> > > > > Some more discussions can be found here:
> > > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > > >
> > > > >
> > > > > >    - Have we considered piggybacking `BeginQuorumEpoch` with the
> `
> > > > > >    FetchQuorumRecords`? I might be missing something obvious but
> I
> > am
> > > > > just
> > > > > >    wondering why don’t we just use the `FindQuorum` and
> > > > > > `FetchQuorumRecords`
> > > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > > >
> > > > >
> > > > > Note that Begin/EndQuorumEpoch is sent from leader -> other voter
> > > > > followers, while FindQuorum / Fetch are sent from follower to
> leader.
> > > > > Arguably one can eventually realize the new leader and epoch via
> > > > gossiping
> > > > > FindQuorum, but that could in practice require a long delay.
> Having a
> > > > > leader -> other voters request helps the new leader epoch to be
> > > > propagated
> > > > > faster under a pull model.
> > > > >
> > > > >
> > > > > >    - And about the `FetchQuorumRecords` response schema, in the
> > > > `Records`
> > > > > >    field of the response, is it just one record or all the
> records
> > > > > starting
> > > > > >    from the FetchOffset? It seems a lot more efficient if we sent
> > all
> > > > the
> > > > > >    records during the bootstrapping of the brokers.
> > > > > >
> > > > >
> > > > > Yes the fetching is batched: FetchOffset is just the starting
> offset
> > of
> > > > the
> > > > > batch of records.
> > > > >
> > > > >
> > > > > >    - Regarding the disruptive broker issues, does our pull based
> > > model
> > > > > >    suffer from it? If so, have we considered the Pre-Vote stage?
> If
> > > > not,
> > > > > > why?
> > > > > >
> > > > > >
> > > > > The disruptive broker is stated in the original Raft paper which is
> > the
> > > > > result of the push model design. Our analysis showed that with the
> > pull
> > > > > model it is no longer an issue.
> > > > >
> > > > >
> > > > > > Thanks a lot for putting this up, and I hope that my questions
> can
> > be
> > > > of
> > > > > > some value to make this KIP better.
> > > > > >
> > > > > > Hope to hear from you soon!
> > > > > >
> > > > > > Best wishes,
> > > > > > Leonard
> > > > > >
> > > > > >
> > > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cmccabe@apache.org
> >
> > > > wrote:
> > > > > >
> > > > > > > Hi Jason,
> > > > > > >
> > > > > > > It's amazing to see this coming together :)
> > > > > > >
> > > > > > > I haven't had a chance to read in detail, but I read the
> outline
> > > and
> > > > a
> > > > > > few
> > > > > > > things jumped out at me.
> > > > > > >
> > > > > > > First, for every epoch that is 32 bits rather than 64, I sort
> of
> > > > wonder
> > > > > > if
> > > > > > > that's a good long-term choice.  I keep reading about stuff
> like
> > > > this:
> > > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
> > Obviously,
> > > > > that
> > > > > > > JIRA is about zxid, which increments much faster than we expect
> > > these
> > > > > > > leader epochs to, but it would still be good to see some rough
> > > > > > calculations
> > > > > > > about how long 32 bits (or really, 31 bits) will last us in the
> > > cases
> > > > > > where
> > > > > > > we're using it, and what the space savings we're getting really
> > is.
> > > > It
> > > > > > > seems like in most cases the tradeoff may not be worth it?
> > > > > > >
> > > > > > > Another thing I've been thinking about is how we do
> > > bootstrapping.  I
> > > > > > > would prefer to be in a world where formatting a new Kafka node
> > > was a
> > > > > > first
> > > > > > > class operation explicitly initiated by the admin, rather than
> > > > > something
> > > > > > > that happened implicitly when you started up the broker and
> > things
> > > > > > "looked
> > > > > > > blank."
> > > > > > >
> > > > > > > The first problem is that things can "look blank" accidentally
> if
> > > the
> > > > > > > storage system is having a bad day.  Clearly in the non-Raft
> > world,
> > > > > this
> > > > > > > leads to data loss if the broker that is (re)started this way
> was
> > > the
> > > > > > > leader for some partitions.
> > > > > > >
> > > > > > > The second problem is that we have a bit of a chicken and egg
> > > problem
> > > > > > with
> > > > > > > certain configuration keys.  For example, maybe you want to
> > > configure
> > > > > > some
> > > > > > > connection security settings in your cluster, but you don't
> want
> > > them
> > > > > to
> > > > > > > ever be stored in a plaintext config file.  (For example, SCRAM
> > > > > > passwords,
> > > > > > > etc.)  You could use a broker API to set the configuration, but
> > > that
> > > > > > brings
> > > > > > > up the chicken and egg problem.  The broker needs to be
> > configured
> > > to
> > > > > > know
> > > > > > > how to talk to you, but you need to configure it before you can
> > > talk
> > > > to
> > > > > > > it.  Using an external secret manager like Vault is one way to
> > > solve
> > > > > > this,
> > > > > > > but not everyone uses an external secret manager.
> > > > > > >
> > > > > > > quorum.voters seems like a similar configuration key.  In the
> > > current
> > > > > > KIP,
> > > > > > > this is only read if there is no other configuration specifying
> > the
> > > > > > quorum
> > > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't need
> this
> > > key
> > > > > > > because we could assume that there was always quorum
> information
> > > > stored
> > > > > > > locally.
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > > > > > > Hi All,
> > > > > > > >
> > > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > > .
> > > > > > > > This proposal specifies a Raft protocol to ultimately replace
> > > > > Zookeeper
> > > > > > > > as
> > > > > > > > documented in KIP-500. Please take a look and share your
> > > thoughts.
> > > > > > > >
> > > > > > > > A few minor notes to set the stage a little bit:
> > > > > > > >
> > > > > > > > - This KIP does not specify the structure of the messages
> used
> > to
> > > > > > > represent
> > > > > > > > metadata in Kafka, nor does it specify the internal API that
> > will
> > > > be
> > > > > > used
> > > > > > > > by the controller. Expect these to come in later proposals.
> > Here
> > > we
> > > > > are
> > > > > > > > primarily concerned with the replication protocol and basic
> > > > > operational
> > > > > > > > mechanics.
> > > > > > > > - We expect many details to change as we get closer to
> > > integration
> > > > > with
> > > > > > > > the controller. Any changes we make will be made either as
> > > > amendments
> > > > > > to
> > > > > > > > this KIP or, in the case of larger changes, as new proposals.
> > > > > > > > - We have a prototype implementation which I will put online
> > > within
> > > > > the
> > > > > > > > next week which may help in understanding some details. It
> has
> > > > > > diverged a
> > > > > > > > little bit from our proposal, so I am taking a little time to
> > > bring
> > > > > it
> > > > > > in
> > > > > > > > line. I'll post an update to this thread when it is available
> > for
> > > > > > review.
> > > > > > > >
> > > > > > > > Finally, I want to mention that this proposal was drafted by
> > > > myself,
> > > > > > > Boyang
> > > > > > > > Chen, and Guozhang Wang.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Jason
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Leonard Ge
> > > > > > Software Engineer Intern - Confluent
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
Hello folks,

I've also updated the KIP wiki page adding a section of the proposed
metrics for this Raft protocol:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP-595:ARaftProtocolfortheMetadataQuorum-Metrics

Please let us know if you have any thoughts about them as well.

Guozhang

On Thu, May 7, 2020 at 4:04 PM Guozhang Wang <wa...@gmail.com> wrote:

> Thanks Jun. Further replies are in-lined.
>
> On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:
>
>> Hi, Guozhang,
>>
>> Thanks for the reply. A few more replies inlined below.
>>
>> On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>> > Hello Jun,
>> >
>> > Thanks for your comments! I'm replying inline below:
>> >
>> > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > 101. Bootstrapping related issues.
>> > > 101.1 Currently, we support auto broker id generation. Is this
>> supported
>> > > for bootstrap brokers?
>> > >
>> >
>> > The vote ids would just be the broker ids. "bootstrap.servers" would be
>> > similar to what client configs have today, where "quorum.voters" would
>> be
>> > pre-defined config values.
>> >
>> >
>> My question was on the auto generated broker id. Currently, the broker can
>> choose to have its broker Id auto generated. The generation is done
>> through
>> ZK to guarantee uniqueness. Without ZK, it's not clear how the broker id
>> is
>> auto generated. "quorum.voters" also can't be set statically if broker ids
>> are auto generated.
>>
>> Jason has explained some ideas that we've discussed so far, the reason we
> intentional did not include them so far is that we feel it is out-side the
> scope of KIP-595. Under the umbrella of KIP-500 we should definitely
> address them though.
>
> On the high-level, our belief is that "joining a quorum" and "joining (or
> more specifically, registering brokers in) the cluster" would be
> de-coupled a bit, where the former should be completed before we do the
> latter. More specifically, assuming the quorum is already up and running,
> after the newly started broker found the leader of the quorum it can send a
> specific RegisterBroker request including its listener / protocol / etc,
> and upon handling it the leader can send back the uniquely generated broker
> id to the new broker, while also executing the "startNewBroker" callback as
> the controller.
>
>
>>
>> > > 102. Log compaction. One weak spot of log compaction is for the
>> consumer
>> > to
>> > > deal with deletes. When a key is deleted, it's retained as a tombstone
>> > > first and then physically removed. If a client misses the tombstone
>> > > (because it's physically removed), it may not be able to update its
>> > > metadata properly. The way we solve this in Kafka is based on a
>> > > configuration (log.cleaner.delete.retention.ms) and we expect a
>> consumer
>> > > having seen an old key to finish reading the deletion tombstone within
>> > that
>> > > time. There is no strong guarantee for that since a broker could be
>> down
>> > > for a long time. It would be better if we can have a more reliable
>> way of
>> > > dealing with deletes.
>> > >
>> >
>> > We propose to capture this in the "FirstDirtyOffset" field of the quorum
>> > record fetch response: the offset is the maximum offset that log
>> compaction
>> > has reached up to. If the follower has fetched beyond this offset it
>> means
>> > itself is safe hence it has seen all records up to that offset. On
>> getting
>> > the response, the follower can then decide if its end offset actually
>> below
>> > that dirty offset (and hence may miss some tombstones). If that's the
>> case:
>> >
>> > 1) Naively, it could re-bootstrap metadata log from the very beginning
>> to
>> > catch up.
>> > 2) During that time, it would refrain itself from answering
>> MetadataRequest
>> > from any clients.
>> >
>> >
>> I am not sure that the "FirstDirtyOffset" field fully addresses the issue.
>> Currently, the deletion tombstone is not removed immediately after a round
>> of cleaning. It's removed after a delay in a subsequent round of cleaning.
>> Consider an example where a key insertion is at offset 200 and a deletion
>> tombstone of the key is at 400. Initially, FirstDirtyOffset is at 300. A
>> follower/observer fetches from offset 0  and fetches the key at offset
>> 200.
>> A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
>> tombstone at 400 is physically removed. The follower/observer continues
>> the
>> fetch, but misses offset 400. It catches all the way to FirstDirtyOffset
>> and declares its metadata as ready. However, its metadata could be stale
>> since it actually misses the deletion of the key.
>>
>> Yeah good question, I should have put more details in my explanation :)
>
> The idea is that we will adjust the log compaction for this raft based
> metadata log: before more details to be explained, since we have two types
> of "watermarks" here, whereas in Kafka the watermark indicates where every
> replica have replicated up to and in Raft the watermark indicates where the
> majority of replicas (here only indicating voters of the quorum, not
> counting observers) have replicated up to, let's call them Kafka watermark
> and Raft watermark. For this special log, we would maintain both watermarks.
>
> When log compacting on the leader, we would only compact up to the Kafka
> watermark, i.e. if there is at least one voter who have not replicated an
> entry, it would not be compacted. The "dirty-offset" is the offset that
> we've compacted up to and is communicated to other voters, and the other
> voters would also compact up to this value --- i.e. the difference here is
> that instead of letting each replica doing log compaction independently,
> we'll have the leader to decide upon which offset to compact to, and
> propagate this value to others to follow, in a more coordinated manner.
> Also note when there are new voters joining the quorum who has not
> replicated up to the dirty-offset, of because of other issues they
> truncated their logs to below the dirty-offset, they'd have to re-bootstrap
> from the beginning, and during this period of time the leader learned about
> this lagging voter would not advance the watermark (also it would not
> decrement it), and hence not compacting either, until the voter(s) has
> caught up to that dirty-offset.
>
> So back to your example above, before the bootstrap voter gets to 300 no
> log compaction would happen on the leader; and until later when the voter
> have got to beyond 400 and hence replicated that tombstone, the log
> compaction would possibly get to that tombstone and remove it. Say later it
> the leader's log compaction reaches 500, it can send this back to the voter
> who can then also compact locally up to 500.
>
>
>> > > 105. Quorum State: In addition to VotedId, do we need the epoch
>> > > corresponding to VotedId? Over time, the same broker Id could be
>> voted in
>> > > different generations with different epoch.
>> > >
>> > >
>> > Hmm, this is a good point. Originally I think the "LeaderEpoch" field in
>> > that file is corresponding to the "latest known leader epoch", not the
>> > "current leader epoch". For example, if the current epoch is N, and
>> then a
>> > vote-request with epoch N+1 is received and the voter granted the vote
>> for
>> > it, then it means for this voter it knows the "latest epoch" is N + 1
>> > although it is unknown if that sending candidate will indeed become the
>> new
>> > leader (which would only be notified via begin-quorum request). However,
>> > when persisting the quorum state, we would encode leader-epoch to N+1,
>> > while the leaderId to be the older leader.
>> >
>> > But now thinking about this a bit more, I feel we should use two
>> separate
>> > epochs, one for the "lates known" and one for the "current" to pair with
>> > the leaderId. I will update the wiki page.
>> >
>> >
>> Hmm, it's kind of weird to bump up the leader epoch before the new leader
>> is actually elected, right.
>>
>>
>> > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to
>> indicate
>> > > that the follower has fetched from an invalid offset and should
>> truncate
>> > to
>> > > the offset/epoch indicated in the response." Observers can't truncate
>> > their
>> > > logs. What should they do with OFFSET_OUT_OF_RANGE?
>> > >
>> > >
>> > I'm not sure if I understand your question? Observers should still be
>> able
>> > to truncate their logs as well.
>> >
>> >
>> Hmm, I thought only the quorum nodes have local logs and observers don't?
>>
>> > 107. "The leader will continue sending BeginQuorumEpoch to each known
>> > voter
>> > > until it has received its endorsement." If a voter is down for a long
>> > time,
>> > > sending BeginQuorumEpoch seems to add unnecessary overhead. Similarly,
>> > if a
>> > > follower stops sending FetchQuorumRecords, does the leader keep
>> sending
>> > > BeginQuorumEpoch?
>> > >
>> >
>> > Regarding BeginQuorumEpoch: that is a good point. The begin-quorum-epoch
>> > request is for voters to quickly get the new leader information; however
>> > even if they do not get them they can still eventually learn about that
>> > from others via gossiping FindQuorum. I think we can adjust the logic to
>> > e.g. exponential back-off or with a limited num.retries.
>> >
>> > Regarding FetchQuorumRecords: if the follower sends FetchQuorumRecords
>> > already, it means that follower already knows that the broker is the
>> > leader, and hence we can stop retrying BeginQuorumEpoch; however it is
>> > possible that after a follower sends FetchQuorumRecords already,
>> suddenly
>> > it stops send it (possibly because it learned about a higher epoch
>> leader),
>> > and hence this broker may be a "zombie" leader and we propose to use the
>> > fetch.timeout to let the leader to try to verify if it has already been
>> > stale.
>> >
>> >
>> It just seems that we should handle these two cases in a consistent way?
>>
>> Yes I agree, on the leader's side, the FetchQuorumRecords from a follower
> could mean that we no longer needs to send BeginQuorumEpoch anymore --- and
> it is already part of our current implementations in
> https://github.com/confluentinc/kafka/commits/kafka-raft
>
>
>> Thanks,
>>
>> Jun
>>
>> >
>> > >
>> > > Jun
>> > >
>> > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com>
>> > wrote:
>> > >
>> > > > Hello Leonard,
>> > > >
>> > > > Thanks for your comments, I'm relying in line below:
>> > > >
>> > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <wge@confluent.io
>> >
>> > > > wrote:
>> > > >
>> > > > > Hi Kafka developers,
>> > > > >
>> > > > > It's great to see this proposal and it took me some time to finish
>> > > > reading
>> > > > > it.
>> > > > >
>> > > > > And I have the following questions about the Proposal:
>> > > > >
>> > > > >    - How do we plan to test this design to ensure its
>> correctness? Or
>> > > > more
>> > > > >    broadly, how do we ensure that our new ‘pull’ based model is
>> > > > functional
>> > > > > and
>> > > > >    correct given that it is different from the original RAFT
>> > > > implementation
>> > > > >    which has formal proof of correctness?
>> > > > >
>> > > >
>> > > > We have two planned verifications on the correctness and liveness of
>> > the
>> > > > design. One is via model verification (TLA+)
>> > > > https://github.com/guozhangwang/kafka-specification
>> > > >
>> > > > Another is via the concurrent simulation tests
>> > > >
>> > > >
>> > >
>> >
>> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
>> > > >
>> > > >    - Have we considered any sensible defaults for the configuration,
>> > i.e.
>> > > > >    all the election timeout, fetch time out, etc.? Or we want to
>> > leave
>> > > > > this to
>> > > > >    a later stage when we do the performance testing, etc.
>> > > > >
>> > > >
>> > > > This is a good question, the reason we did not set any default
>> values
>> > for
>> > > > the timeout configurations is that we think it may take some
>> > benchmarking
>> > > > experiments to get these defaults right. Some high-level principles
>> to
>> > > > consider: 1) the fetch.timeout should be around the same scale with
>> zk
>> > > > session timeout, which is now 18 seconds by default -- in practice
>> > we've
>> > > > seen unstable networks having more than 10 secs of transient
>> > > connectivity,
>> > > > 2) the election.timeout, however, should be smaller than the fetch
>> > > timeout
>> > > > as is also suggested as a practical optimization in literature:
>> > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
>> > > >
>> > > > Some more discussions can be found here:
>> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
>> > > >
>> > > >
>> > > > >    - Have we considered piggybacking `BeginQuorumEpoch` with the `
>> > > > >    FetchQuorumRecords`? I might be missing something obvious but
>> I am
>> > > > just
>> > > > >    wondering why don’t we just use the `FindQuorum` and
>> > > > > `FetchQuorumRecords`
>> > > > >    APIs and remove the `BeginQuorumEpoch` API?
>> > > > >
>> > > >
>> > > > Note that Begin/EndQuorumEpoch is sent from leader -> other voter
>> > > > followers, while FindQuorum / Fetch are sent from follower to
>> leader.
>> > > > Arguably one can eventually realize the new leader and epoch via
>> > > gossiping
>> > > > FindQuorum, but that could in practice require a long delay. Having
>> a
>> > > > leader -> other voters request helps the new leader epoch to be
>> > > propagated
>> > > > faster under a pull model.
>> > > >
>> > > >
>> > > > >    - And about the `FetchQuorumRecords` response schema, in the
>> > > `Records`
>> > > > >    field of the response, is it just one record or all the records
>> > > > starting
>> > > > >    from the FetchOffset? It seems a lot more efficient if we sent
>> all
>> > > the
>> > > > >    records during the bootstrapping of the brokers.
>> > > > >
>> > > >
>> > > > Yes the fetching is batched: FetchOffset is just the starting
>> offset of
>> > > the
>> > > > batch of records.
>> > > >
>> > > >
>> > > > >    - Regarding the disruptive broker issues, does our pull based
>> > model
>> > > > >    suffer from it? If so, have we considered the Pre-Vote stage?
>> If
>> > > not,
>> > > > > why?
>> > > > >
>> > > > >
>> > > > The disruptive broker is stated in the original Raft paper which is
>> the
>> > > > result of the push model design. Our analysis showed that with the
>> pull
>> > > > model it is no longer an issue.
>> > > >
>> > > >
>> > > > > Thanks a lot for putting this up, and I hope that my questions
>> can be
>> > > of
>> > > > > some value to make this KIP better.
>> > > > >
>> > > > > Hope to hear from you soon!
>> > > > >
>> > > > > Best wishes,
>> > > > > Leonard
>> > > > >
>> > > > >
>> > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org>
>> > > wrote:
>> > > > >
>> > > > > > Hi Jason,
>> > > > > >
>> > > > > > It's amazing to see this coming together :)
>> > > > > >
>> > > > > > I haven't had a chance to read in detail, but I read the outline
>> > and
>> > > a
>> > > > > few
>> > > > > > things jumped out at me.
>> > > > > >
>> > > > > > First, for every epoch that is 32 bits rather than 64, I sort of
>> > > wonder
>> > > > > if
>> > > > > > that's a good long-term choice.  I keep reading about stuff like
>> > > this:
>> > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
>> Obviously,
>> > > > that
>> > > > > > JIRA is about zxid, which increments much faster than we expect
>> > these
>> > > > > > leader epochs to, but it would still be good to see some rough
>> > > > > calculations
>> > > > > > about how long 32 bits (or really, 31 bits) will last us in the
>> > cases
>> > > > > where
>> > > > > > we're using it, and what the space savings we're getting really
>> is.
>> > > It
>> > > > > > seems like in most cases the tradeoff may not be worth it?
>> > > > > >
>> > > > > > Another thing I've been thinking about is how we do
>> > bootstrapping.  I
>> > > > > > would prefer to be in a world where formatting a new Kafka node
>> > was a
>> > > > > first
>> > > > > > class operation explicitly initiated by the admin, rather than
>> > > > something
>> > > > > > that happened implicitly when you started up the broker and
>> things
>> > > > > "looked
>> > > > > > blank."
>> > > > > >
>> > > > > > The first problem is that things can "look blank" accidentally
>> if
>> > the
>> > > > > > storage system is having a bad day.  Clearly in the non-Raft
>> world,
>> > > > this
>> > > > > > leads to data loss if the broker that is (re)started this way
>> was
>> > the
>> > > > > > leader for some partitions.
>> > > > > >
>> > > > > > The second problem is that we have a bit of a chicken and egg
>> > problem
>> > > > > with
>> > > > > > certain configuration keys.  For example, maybe you want to
>> > configure
>> > > > > some
>> > > > > > connection security settings in your cluster, but you don't want
>> > them
>> > > > to
>> > > > > > ever be stored in a plaintext config file.  (For example, SCRAM
>> > > > > passwords,
>> > > > > > etc.)  You could use a broker API to set the configuration, but
>> > that
>> > > > > brings
>> > > > > > up the chicken and egg problem.  The broker needs to be
>> configured
>> > to
>> > > > > know
>> > > > > > how to talk to you, but you need to configure it before you can
>> > talk
>> > > to
>> > > > > > it.  Using an external secret manager like Vault is one way to
>> > solve
>> > > > > this,
>> > > > > > but not everyone uses an external secret manager.
>> > > > > >
>> > > > > > quorum.voters seems like a similar configuration key.  In the
>> > current
>> > > > > KIP,
>> > > > > > this is only read if there is no other configuration specifying
>> the
>> > > > > quorum
>> > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't need
>> this
>> > key
>> > > > > > because we could assume that there was always quorum information
>> > > stored
>> > > > > > locally.
>> > > > > >
>> > > > > > best,
>> > > > > > Colin
>> > > > > >
>> > > > > >
>> > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
>> > > > > > > Hi All,
>> > > > > > >
>> > > > > > > I'd like to start a discussion on KIP-595:
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
>> > > > > > .
>> > > > > > > This proposal specifies a Raft protocol to ultimately replace
>> > > > Zookeeper
>> > > > > > > as
>> > > > > > > documented in KIP-500. Please take a look and share your
>> > thoughts.
>> > > > > > >
>> > > > > > > A few minor notes to set the stage a little bit:
>> > > > > > >
>> > > > > > > - This KIP does not specify the structure of the messages
>> used to
>> > > > > > represent
>> > > > > > > metadata in Kafka, nor does it specify the internal API that
>> will
>> > > be
>> > > > > used
>> > > > > > > by the controller. Expect these to come in later proposals.
>> Here
>> > we
>> > > > are
>> > > > > > > primarily concerned with the replication protocol and basic
>> > > > operational
>> > > > > > > mechanics.
>> > > > > > > - We expect many details to change as we get closer to
>> > integration
>> > > > with
>> > > > > > > the controller. Any changes we make will be made either as
>> > > amendments
>> > > > > to
>> > > > > > > this KIP or, in the case of larger changes, as new proposals.
>> > > > > > > - We have a prototype implementation which I will put online
>> > within
>> > > > the
>> > > > > > > next week which may help in understanding some details. It has
>> > > > > diverged a
>> > > > > > > little bit from our proposal, so I am taking a little time to
>> > bring
>> > > > it
>> > > > > in
>> > > > > > > line. I'll post an update to this thread when it is available
>> for
>> > > > > review.
>> > > > > > >
>> > > > > > > Finally, I want to mention that this proposal was drafted by
>> > > myself,
>> > > > > > Boyang
>> > > > > > > Chen, and Guozhang Wang.
>> > > > > > >
>> > > > > > > Thanks,
>> > > > > > > Jason
>> > > > > > >
>> > > > > >
>> > > > >
>> > > > >
>> > > > > --
>> > > > > Leonard Ge
>> > > > > Software Engineer Intern - Confluent
>> > > > >
>> > > >
>> > > >
>> > > > --
>> > > > -- Guozhang
>> > > >
>> > >
>> >
>> >
>> > --
>> > -- Guozhang
>> >
>>
>
>
> --
> -- Guozhang
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
Thanks Jun. Further replies are in-lined.

On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:

> Hi, Guozhang,
>
> Thanks for the reply. A few more replies inlined below.
>
> On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hello Jun,
> >
> > Thanks for your comments! I'm replying inline below:
> >
> > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > > 101. Bootstrapping related issues.
> > > 101.1 Currently, we support auto broker id generation. Is this
> supported
> > > for bootstrap brokers?
> > >
> >
> > The vote ids would just be the broker ids. "bootstrap.servers" would be
> > similar to what client configs have today, where "quorum.voters" would be
> > pre-defined config values.
> >
> >
> My question was on the auto generated broker id. Currently, the broker can
> choose to have its broker Id auto generated. The generation is done through
> ZK to guarantee uniqueness. Without ZK, it's not clear how the broker id is
> auto generated. "quorum.voters" also can't be set statically if broker ids
> are auto generated.
>
> Jason has explained some ideas that we've discussed so far, the reason we
intentional did not include them so far is that we feel it is out-side the
scope of KIP-595. Under the umbrella of KIP-500 we should definitely
address them though.

On the high-level, our belief is that "joining a quorum" and "joining (or
more specifically, registering brokers in) the cluster" would be
de-coupled a bit, where the former should be completed before we do the
latter. More specifically, assuming the quorum is already up and running,
after the newly started broker found the leader of the quorum it can send a
specific RegisterBroker request including its listener / protocol / etc,
and upon handling it the leader can send back the uniquely generated broker
id to the new broker, while also executing the "startNewBroker" callback as
the controller.


>
> > > 102. Log compaction. One weak spot of log compaction is for the
> consumer
> > to
> > > deal with deletes. When a key is deleted, it's retained as a tombstone
> > > first and then physically removed. If a client misses the tombstone
> > > (because it's physically removed), it may not be able to update its
> > > metadata properly. The way we solve this in Kafka is based on a
> > > configuration (log.cleaner.delete.retention.ms) and we expect a
> consumer
> > > having seen an old key to finish reading the deletion tombstone within
> > that
> > > time. There is no strong guarantee for that since a broker could be
> down
> > > for a long time. It would be better if we can have a more reliable way
> of
> > > dealing with deletes.
> > >
> >
> > We propose to capture this in the "FirstDirtyOffset" field of the quorum
> > record fetch response: the offset is the maximum offset that log
> compaction
> > has reached up to. If the follower has fetched beyond this offset it
> means
> > itself is safe hence it has seen all records up to that offset. On
> getting
> > the response, the follower can then decide if its end offset actually
> below
> > that dirty offset (and hence may miss some tombstones). If that's the
> case:
> >
> > 1) Naively, it could re-bootstrap metadata log from the very beginning to
> > catch up.
> > 2) During that time, it would refrain itself from answering
> MetadataRequest
> > from any clients.
> >
> >
> I am not sure that the "FirstDirtyOffset" field fully addresses the issue.
> Currently, the deletion tombstone is not removed immediately after a round
> of cleaning. It's removed after a delay in a subsequent round of cleaning.
> Consider an example where a key insertion is at offset 200 and a deletion
> tombstone of the key is at 400. Initially, FirstDirtyOffset is at 300. A
> follower/observer fetches from offset 0  and fetches the key at offset 200.
> A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
> tombstone at 400 is physically removed. The follower/observer continues the
> fetch, but misses offset 400. It catches all the way to FirstDirtyOffset
> and declares its metadata as ready. However, its metadata could be stale
> since it actually misses the deletion of the key.
>
> Yeah good question, I should have put more details in my explanation :)

The idea is that we will adjust the log compaction for this raft based
metadata log: before more details to be explained, since we have two types
of "watermarks" here, whereas in Kafka the watermark indicates where every
replica have replicated up to and in Raft the watermark indicates where the
majority of replicas (here only indicating voters of the quorum, not
counting observers) have replicated up to, let's call them Kafka watermark
and Raft watermark. For this special log, we would maintain both watermarks.

When log compacting on the leader, we would only compact up to the Kafka
watermark, i.e. if there is at least one voter who have not replicated an
entry, it would not be compacted. The "dirty-offset" is the offset that
we've compacted up to and is communicated to other voters, and the other
voters would also compact up to this value --- i.e. the difference here is
that instead of letting each replica doing log compaction independently,
we'll have the leader to decide upon which offset to compact to, and
propagate this value to others to follow, in a more coordinated manner.
Also note when there are new voters joining the quorum who has not
replicated up to the dirty-offset, of because of other issues they
truncated their logs to below the dirty-offset, they'd have to re-bootstrap
from the beginning, and during this period of time the leader learned about
this lagging voter would not advance the watermark (also it would not
decrement it), and hence not compacting either, until the voter(s) has
caught up to that dirty-offset.

So back to your example above, before the bootstrap voter gets to 300 no
log compaction would happen on the leader; and until later when the voter
have got to beyond 400 and hence replicated that tombstone, the log
compaction would possibly get to that tombstone and remove it. Say later it
the leader's log compaction reaches 500, it can send this back to the voter
who can then also compact locally up to 500.


> > > 105. Quorum State: In addition to VotedId, do we need the epoch
> > > corresponding to VotedId? Over time, the same broker Id could be voted
> in
> > > different generations with different epoch.
> > >
> > >
> > Hmm, this is a good point. Originally I think the "LeaderEpoch" field in
> > that file is corresponding to the "latest known leader epoch", not the
> > "current leader epoch". For example, if the current epoch is N, and then
> a
> > vote-request with epoch N+1 is received and the voter granted the vote
> for
> > it, then it means for this voter it knows the "latest epoch" is N + 1
> > although it is unknown if that sending candidate will indeed become the
> new
> > leader (which would only be notified via begin-quorum request). However,
> > when persisting the quorum state, we would encode leader-epoch to N+1,
> > while the leaderId to be the older leader.
> >
> > But now thinking about this a bit more, I feel we should use two separate
> > epochs, one for the "lates known" and one for the "current" to pair with
> > the leaderId. I will update the wiki page.
> >
> >
> Hmm, it's kind of weird to bump up the leader epoch before the new leader
> is actually elected, right.
>
>
> > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to
> indicate
> > > that the follower has fetched from an invalid offset and should
> truncate
> > to
> > > the offset/epoch indicated in the response." Observers can't truncate
> > their
> > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > >
> > >
> > I'm not sure if I understand your question? Observers should still be
> able
> > to truncate their logs as well.
> >
> >
> Hmm, I thought only the quorum nodes have local logs and observers don't?
>
> > 107. "The leader will continue sending BeginQuorumEpoch to each known
> > voter
> > > until it has received its endorsement." If a voter is down for a long
> > time,
> > > sending BeginQuorumEpoch seems to add unnecessary overhead. Similarly,
> > if a
> > > follower stops sending FetchQuorumRecords, does the leader keep sending
> > > BeginQuorumEpoch?
> > >
> >
> > Regarding BeginQuorumEpoch: that is a good point. The begin-quorum-epoch
> > request is for voters to quickly get the new leader information; however
> > even if they do not get them they can still eventually learn about that
> > from others via gossiping FindQuorum. I think we can adjust the logic to
> > e.g. exponential back-off or with a limited num.retries.
> >
> > Regarding FetchQuorumRecords: if the follower sends FetchQuorumRecords
> > already, it means that follower already knows that the broker is the
> > leader, and hence we can stop retrying BeginQuorumEpoch; however it is
> > possible that after a follower sends FetchQuorumRecords already, suddenly
> > it stops send it (possibly because it learned about a higher epoch
> leader),
> > and hence this broker may be a "zombie" leader and we propose to use the
> > fetch.timeout to let the leader to try to verify if it has already been
> > stale.
> >
> >
> It just seems that we should handle these two cases in a consistent way?
>
> Yes I agree, on the leader's side, the FetchQuorumRecords from a follower
could mean that we no longer needs to send BeginQuorumEpoch anymore --- and
it is already part of our current implementations in
https://github.com/confluentinc/kafka/commits/kafka-raft


> Thanks,
>
> Jun
>
> >
> > >
> > > Jun
> > >
> > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hello Leonard,
> > > >
> > > > Thanks for your comments, I'm relying in line below:
> > > >
> > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <wg...@confluent.io>
> > > > wrote:
> > > >
> > > > > Hi Kafka developers,
> > > > >
> > > > > It's great to see this proposal and it took me some time to finish
> > > > reading
> > > > > it.
> > > > >
> > > > > And I have the following questions about the Proposal:
> > > > >
> > > > >    - How do we plan to test this design to ensure its correctness?
> Or
> > > > more
> > > > >    broadly, how do we ensure that our new ‘pull’ based model is
> > > > functional
> > > > > and
> > > > >    correct given that it is different from the original RAFT
> > > > implementation
> > > > >    which has formal proof of correctness?
> > > > >
> > > >
> > > > We have two planned verifications on the correctness and liveness of
> > the
> > > > design. One is via model verification (TLA+)
> > > > https://github.com/guozhangwang/kafka-specification
> > > >
> > > > Another is via the concurrent simulation tests
> > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > >
> > > >    - Have we considered any sensible defaults for the configuration,
> > i.e.
> > > > >    all the election timeout, fetch time out, etc.? Or we want to
> > leave
> > > > > this to
> > > > >    a later stage when we do the performance testing, etc.
> > > > >
> > > >
> > > > This is a good question, the reason we did not set any default values
> > for
> > > > the timeout configurations is that we think it may take some
> > benchmarking
> > > > experiments to get these defaults right. Some high-level principles
> to
> > > > consider: 1) the fetch.timeout should be around the same scale with
> zk
> > > > session timeout, which is now 18 seconds by default -- in practice
> > we've
> > > > seen unstable networks having more than 10 secs of transient
> > > connectivity,
> > > > 2) the election.timeout, however, should be smaller than the fetch
> > > timeout
> > > > as is also suggested as a practical optimization in literature:
> > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > >
> > > > Some more discussions can be found here:
> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > >
> > > >
> > > > >    - Have we considered piggybacking `BeginQuorumEpoch` with the `
> > > > >    FetchQuorumRecords`? I might be missing something obvious but I
> am
> > > > just
> > > > >    wondering why don’t we just use the `FindQuorum` and
> > > > > `FetchQuorumRecords`
> > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > >
> > > >
> > > > Note that Begin/EndQuorumEpoch is sent from leader -> other voter
> > > > followers, while FindQuorum / Fetch are sent from follower to leader.
> > > > Arguably one can eventually realize the new leader and epoch via
> > > gossiping
> > > > FindQuorum, but that could in practice require a long delay. Having a
> > > > leader -> other voters request helps the new leader epoch to be
> > > propagated
> > > > faster under a pull model.
> > > >
> > > >
> > > > >    - And about the `FetchQuorumRecords` response schema, in the
> > > `Records`
> > > > >    field of the response, is it just one record or all the records
> > > > starting
> > > > >    from the FetchOffset? It seems a lot more efficient if we sent
> all
> > > the
> > > > >    records during the bootstrapping of the brokers.
> > > > >
> > > >
> > > > Yes the fetching is batched: FetchOffset is just the starting offset
> of
> > > the
> > > > batch of records.
> > > >
> > > >
> > > > >    - Regarding the disruptive broker issues, does our pull based
> > model
> > > > >    suffer from it? If so, have we considered the Pre-Vote stage? If
> > > not,
> > > > > why?
> > > > >
> > > > >
> > > > The disruptive broker is stated in the original Raft paper which is
> the
> > > > result of the push model design. Our analysis showed that with the
> pull
> > > > model it is no longer an issue.
> > > >
> > > >
> > > > > Thanks a lot for putting this up, and I hope that my questions can
> be
> > > of
> > > > > some value to make this KIP better.
> > > > >
> > > > > Hope to hear from you soon!
> > > > >
> > > > > Best wishes,
> > > > > Leonard
> > > > >
> > > > >
> > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org>
> > > wrote:
> > > > >
> > > > > > Hi Jason,
> > > > > >
> > > > > > It's amazing to see this coming together :)
> > > > > >
> > > > > > I haven't had a chance to read in detail, but I read the outline
> > and
> > > a
> > > > > few
> > > > > > things jumped out at me.
> > > > > >
> > > > > > First, for every epoch that is 32 bits rather than 64, I sort of
> > > wonder
> > > > > if
> > > > > > that's a good long-term choice.  I keep reading about stuff like
> > > this:
> > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
> Obviously,
> > > > that
> > > > > > JIRA is about zxid, which increments much faster than we expect
> > these
> > > > > > leader epochs to, but it would still be good to see some rough
> > > > > calculations
> > > > > > about how long 32 bits (or really, 31 bits) will last us in the
> > cases
> > > > > where
> > > > > > we're using it, and what the space savings we're getting really
> is.
> > > It
> > > > > > seems like in most cases the tradeoff may not be worth it?
> > > > > >
> > > > > > Another thing I've been thinking about is how we do
> > bootstrapping.  I
> > > > > > would prefer to be in a world where formatting a new Kafka node
> > was a
> > > > > first
> > > > > > class operation explicitly initiated by the admin, rather than
> > > > something
> > > > > > that happened implicitly when you started up the broker and
> things
> > > > > "looked
> > > > > > blank."
> > > > > >
> > > > > > The first problem is that things can "look blank" accidentally if
> > the
> > > > > > storage system is having a bad day.  Clearly in the non-Raft
> world,
> > > > this
> > > > > > leads to data loss if the broker that is (re)started this way was
> > the
> > > > > > leader for some partitions.
> > > > > >
> > > > > > The second problem is that we have a bit of a chicken and egg
> > problem
> > > > > with
> > > > > > certain configuration keys.  For example, maybe you want to
> > configure
> > > > > some
> > > > > > connection security settings in your cluster, but you don't want
> > them
> > > > to
> > > > > > ever be stored in a plaintext config file.  (For example, SCRAM
> > > > > passwords,
> > > > > > etc.)  You could use a broker API to set the configuration, but
> > that
> > > > > brings
> > > > > > up the chicken and egg problem.  The broker needs to be
> configured
> > to
> > > > > know
> > > > > > how to talk to you, but you need to configure it before you can
> > talk
> > > to
> > > > > > it.  Using an external secret manager like Vault is one way to
> > solve
> > > > > this,
> > > > > > but not everyone uses an external secret manager.
> > > > > >
> > > > > > quorum.voters seems like a similar configuration key.  In the
> > current
> > > > > KIP,
> > > > > > this is only read if there is no other configuration specifying
> the
> > > > > quorum
> > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't need this
> > key
> > > > > > because we could assume that there was always quorum information
> > > stored
> > > > > > locally.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > >
> > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > > > > > Hi All,
> > > > > > >
> > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > .
> > > > > > > This proposal specifies a Raft protocol to ultimately replace
> > > > Zookeeper
> > > > > > > as
> > > > > > > documented in KIP-500. Please take a look and share your
> > thoughts.
> > > > > > >
> > > > > > > A few minor notes to set the stage a little bit:
> > > > > > >
> > > > > > > - This KIP does not specify the structure of the messages used
> to
> > > > > > represent
> > > > > > > metadata in Kafka, nor does it specify the internal API that
> will
> > > be
> > > > > used
> > > > > > > by the controller. Expect these to come in later proposals.
> Here
> > we
> > > > are
> > > > > > > primarily concerned with the replication protocol and basic
> > > > operational
> > > > > > > mechanics.
> > > > > > > - We expect many details to change as we get closer to
> > integration
> > > > with
> > > > > > > the controller. Any changes we make will be made either as
> > > amendments
> > > > > to
> > > > > > > this KIP or, in the case of larger changes, as new proposals.
> > > > > > > - We have a prototype implementation which I will put online
> > within
> > > > the
> > > > > > > next week which may help in understanding some details. It has
> > > > > diverged a
> > > > > > > little bit from our proposal, so I am taking a little time to
> > bring
> > > > it
> > > > > in
> > > > > > > line. I'll post an update to this thread when it is available
> for
> > > > > review.
> > > > > > >
> > > > > > > Finally, I want to mention that this proposal was drafted by
> > > myself,
> > > > > > Boyang
> > > > > > > Chen, and Guozhang Wang.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Jason
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Leonard Ge
> > > > > Software Engineer Intern - Confluent
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Jun,

Thanks for the comments. Here are a few quick responses.

> My question was on the auto generated broker id. Currently, the broker can
choose to have its broker Id auto generated. The generation is done through
ZK to guarantee uniqueness. Without ZK, it's not clear how the broker id is
auto generated. "quorum.voters" also can't be set statically if broker ids
are auto generated.

Yeah, I'm not sure I see a way for this to be supported during the initial
bootstrapping of the cluster. I had hoped we could leave it for a follow-up
proposal, but I think we need to have a section on broker registration
since there are some details to work out. Auto-generation of ids is one of
them. Another is the mechanism for registering broker state and how we
track liveness.

> Hmm, it's kind of weird to bump up the leader epoch before the new leader
is actually elected, right.

That is what the Raft protocol specifies. We bump the epoch upon becoming a
candidate. Once the epoch is bumped, we cannot go back to a previous epoch,
so there is no value remembering it. Perhaps it would be less confusing if
we used "epoch" instead of "leader epoch" consistently.

> Hmm, I thought only the quorum nodes have local logs and observers don't?

Observers also replicate and store the log.

Thanks,
Jason

On Mon, May 4, 2020 at 11:58 AM Jun Rao <ju...@confluent.io> wrote:

> Hi, Guozhang,
>
> Thanks for the reply. A few more replies inlined below.
>
> On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hello Jun,
> >
> > Thanks for your comments! I'm replying inline below:
> >
> > On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > > 101. Bootstrapping related issues.
> > > 101.1 Currently, we support auto broker id generation. Is this
> supported
> > > for bootstrap brokers?
> > >
> >
> > The vote ids would just be the broker ids. "bootstrap.servers" would be
> > similar to what client configs have today, where "quorum.voters" would be
> > pre-defined config values.
> >
> >
> My question was on the auto generated broker id. Currently, the broker can
> choose to have its broker Id auto generated. The generation is done through
> ZK to guarantee uniqueness. Without ZK, it's not clear how the broker id is
> auto generated. "quorum.voters" also can't be set statically if broker ids
> are auto generated.
>
>
> > > 102. Log compaction. One weak spot of log compaction is for the
> consumer
> > to
> > > deal with deletes. When a key is deleted, it's retained as a tombstone
> > > first and then physically removed. If a client misses the tombstone
> > > (because it's physically removed), it may not be able to update its
> > > metadata properly. The way we solve this in Kafka is based on a
> > > configuration (log.cleaner.delete.retention.ms) and we expect a
> consumer
> > > having seen an old key to finish reading the deletion tombstone within
> > that
> > > time. There is no strong guarantee for that since a broker could be
> down
> > > for a long time. It would be better if we can have a more reliable way
> of
> > > dealing with deletes.
> > >
> >
> > We propose to capture this in the "FirstDirtyOffset" field of the quorum
> > record fetch response: the offset is the maximum offset that log
> compaction
> > has reached up to. If the follower has fetched beyond this offset it
> means
> > itself is safe hence it has seen all records up to that offset. On
> getting
> > the response, the follower can then decide if its end offset actually
> below
> > that dirty offset (and hence may miss some tombstones). If that's the
> case:
> >
> > 1) Naively, it could re-bootstrap metadata log from the very beginning to
> > catch up.
> > 2) During that time, it would refrain itself from answering
> MetadataRequest
> > from any clients.
> >
> >
> I am not sure that the "FirstDirtyOffset" field fully addresses the issue.
> Currently, the deletion tombstone is not removed immediately after a round
> of cleaning. It's removed after a delay in a subsequent round of cleaning.
> Consider an example where a key insertion is at offset 200 and a deletion
> tombstone of the key is at 400. Initially, FirstDirtyOffset is at 300. A
> follower/observer fetches from offset 0  and fetches the key at offset 200.
> A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
> tombstone at 400 is physically removed. The follower/observer continues the
> fetch, but misses offset 400. It catches all the way to FirstDirtyOffset
> and declares its metadata as ready. However, its metadata could be stale
> since it actually misses the deletion of the key.
>
>
> > > 105. Quorum State: In addition to VotedId, do we need the epoch
> > > corresponding to VotedId? Over time, the same broker Id could be voted
> in
> > > different generations with different epoch.
> > >
> > >
> > Hmm, this is a good point. Originally I think the "LeaderEpoch" field in
> > that file is corresponding to the "latest known leader epoch", not the
> > "current leader epoch". For example, if the current epoch is N, and then
> a
> > vote-request with epoch N+1 is received and the voter granted the vote
> for
> > it, then it means for this voter it knows the "latest epoch" is N + 1
> > although it is unknown if that sending candidate will indeed become the
> new
> > leader (which would only be notified via begin-quorum request). However,
> > when persisting the quorum state, we would encode leader-epoch to N+1,
> > while the leaderId to be the older leader.
> >
> > But now thinking about this a bit more, I feel we should use two separate
> > epochs, one for the "lates known" and one for the "current" to pair with
> > the leaderId. I will update the wiki page.
> >
> >
> Hmm, it's kind of weird to bump up the leader epoch before the new leader
> is actually elected, right.
>
>
> > > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to
> indicate
> > > that the follower has fetched from an invalid offset and should
> truncate
> > to
> > > the offset/epoch indicated in the response." Observers can't truncate
> > their
> > > logs. What should they do with OFFSET_OUT_OF_RANGE?
> > >
> > >
> > I'm not sure if I understand your question? Observers should still be
> able
> > to truncate their logs as well.
> >
> >
> Hmm, I thought only the quorum nodes have local logs and observers don't?
>
> > 107. "The leader will continue sending BeginQuorumEpoch to each known
> > voter
> > > until it has received its endorsement." If a voter is down for a long
> > time,
> > > sending BeginQuorumEpoch seems to add unnecessary overhead. Similarly,
> > if a
> > > follower stops sending FetchQuorumRecords, does the leader keep sending
> > > BeginQuorumEpoch?
> > >
> >
> > Regarding BeginQuorumEpoch: that is a good point. The begin-quorum-epoch
> > request is for voters to quickly get the new leader information; however
> > even if they do not get them they can still eventually learn about that
> > from others via gossiping FindQuorum. I think we can adjust the logic to
> > e.g. exponential back-off or with a limited num.retries.
> >
> > Regarding FetchQuorumRecords: if the follower sends FetchQuorumRecords
> > already, it means that follower already knows that the broker is the
> > leader, and hence we can stop retrying BeginQuorumEpoch; however it is
> > possible that after a follower sends FetchQuorumRecords already, suddenly
> > it stops send it (possibly because it learned about a higher epoch
> leader),
> > and hence this broker may be a "zombie" leader and we propose to use the
> > fetch.timeout to let the leader to try to verify if it has already been
> > stale.
> >
> >
> It just seems that we should handle these two cases in a consistent way?
>
> Thanks,
>
> Jun
>
> >
> > >
> > > Jun
> > >
> > > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hello Leonard,
> > > >
> > > > Thanks for your comments, I'm relying in line below:
> > > >
> > > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <wg...@confluent.io>
> > > > wrote:
> > > >
> > > > > Hi Kafka developers,
> > > > >
> > > > > It's great to see this proposal and it took me some time to finish
> > > > reading
> > > > > it.
> > > > >
> > > > > And I have the following questions about the Proposal:
> > > > >
> > > > >    - How do we plan to test this design to ensure its correctness?
> Or
> > > > more
> > > > >    broadly, how do we ensure that our new ‘pull’ based model is
> > > > functional
> > > > > and
> > > > >    correct given that it is different from the original RAFT
> > > > implementation
> > > > >    which has formal proof of correctness?
> > > > >
> > > >
> > > > We have two planned verifications on the correctness and liveness of
> > the
> > > > design. One is via model verification (TLA+)
> > > > https://github.com/guozhangwang/kafka-specification
> > > >
> > > > Another is via the concurrent simulation tests
> > > >
> > > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > > >
> > > >    - Have we considered any sensible defaults for the configuration,
> > i.e.
> > > > >    all the election timeout, fetch time out, etc.? Or we want to
> > leave
> > > > > this to
> > > > >    a later stage when we do the performance testing, etc.
> > > > >
> > > >
> > > > This is a good question, the reason we did not set any default values
> > for
> > > > the timeout configurations is that we think it may take some
> > benchmarking
> > > > experiments to get these defaults right. Some high-level principles
> to
> > > > consider: 1) the fetch.timeout should be around the same scale with
> zk
> > > > session timeout, which is now 18 seconds by default -- in practice
> > we've
> > > > seen unstable networks having more than 10 secs of transient
> > > connectivity,
> > > > 2) the election.timeout, however, should be smaller than the fetch
> > > timeout
> > > > as is also suggested as a practical optimization in literature:
> > > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > > >
> > > > Some more discussions can be found here:
> > > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > > >
> > > >
> > > > >    - Have we considered piggybacking `BeginQuorumEpoch` with the `
> > > > >    FetchQuorumRecords`? I might be missing something obvious but I
> am
> > > > just
> > > > >    wondering why don’t we just use the `FindQuorum` and
> > > > > `FetchQuorumRecords`
> > > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > > >
> > > >
> > > > Note that Begin/EndQuorumEpoch is sent from leader -> other voter
> > > > followers, while FindQuorum / Fetch are sent from follower to leader.
> > > > Arguably one can eventually realize the new leader and epoch via
> > > gossiping
> > > > FindQuorum, but that could in practice require a long delay. Having a
> > > > leader -> other voters request helps the new leader epoch to be
> > > propagated
> > > > faster under a pull model.
> > > >
> > > >
> > > > >    - And about the `FetchQuorumRecords` response schema, in the
> > > `Records`
> > > > >    field of the response, is it just one record or all the records
> > > > starting
> > > > >    from the FetchOffset? It seems a lot more efficient if we sent
> all
> > > the
> > > > >    records during the bootstrapping of the brokers.
> > > > >
> > > >
> > > > Yes the fetching is batched: FetchOffset is just the starting offset
> of
> > > the
> > > > batch of records.
> > > >
> > > >
> > > > >    - Regarding the disruptive broker issues, does our pull based
> > model
> > > > >    suffer from it? If so, have we considered the Pre-Vote stage? If
> > > not,
> > > > > why?
> > > > >
> > > > >
> > > > The disruptive broker is stated in the original Raft paper which is
> the
> > > > result of the push model design. Our analysis showed that with the
> pull
> > > > model it is no longer an issue.
> > > >
> > > >
> > > > > Thanks a lot for putting this up, and I hope that my questions can
> be
> > > of
> > > > > some value to make this KIP better.
> > > > >
> > > > > Hope to hear from you soon!
> > > > >
> > > > > Best wishes,
> > > > > Leonard
> > > > >
> > > > >
> > > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org>
> > > wrote:
> > > > >
> > > > > > Hi Jason,
> > > > > >
> > > > > > It's amazing to see this coming together :)
> > > > > >
> > > > > > I haven't had a chance to read in detail, but I read the outline
> > and
> > > a
> > > > > few
> > > > > > things jumped out at me.
> > > > > >
> > > > > > First, for every epoch that is 32 bits rather than 64, I sort of
> > > wonder
> > > > > if
> > > > > > that's a good long-term choice.  I keep reading about stuff like
> > > this:
> > > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .
> Obviously,
> > > > that
> > > > > > JIRA is about zxid, which increments much faster than we expect
> > these
> > > > > > leader epochs to, but it would still be good to see some rough
> > > > > calculations
> > > > > > about how long 32 bits (or really, 31 bits) will last us in the
> > cases
> > > > > where
> > > > > > we're using it, and what the space savings we're getting really
> is.
> > > It
> > > > > > seems like in most cases the tradeoff may not be worth it?
> > > > > >
> > > > > > Another thing I've been thinking about is how we do
> > bootstrapping.  I
> > > > > > would prefer to be in a world where formatting a new Kafka node
> > was a
> > > > > first
> > > > > > class operation explicitly initiated by the admin, rather than
> > > > something
> > > > > > that happened implicitly when you started up the broker and
> things
> > > > > "looked
> > > > > > blank."
> > > > > >
> > > > > > The first problem is that things can "look blank" accidentally if
> > the
> > > > > > storage system is having a bad day.  Clearly in the non-Raft
> world,
> > > > this
> > > > > > leads to data loss if the broker that is (re)started this way was
> > the
> > > > > > leader for some partitions.
> > > > > >
> > > > > > The second problem is that we have a bit of a chicken and egg
> > problem
> > > > > with
> > > > > > certain configuration keys.  For example, maybe you want to
> > configure
> > > > > some
> > > > > > connection security settings in your cluster, but you don't want
> > them
> > > > to
> > > > > > ever be stored in a plaintext config file.  (For example, SCRAM
> > > > > passwords,
> > > > > > etc.)  You could use a broker API to set the configuration, but
> > that
> > > > > brings
> > > > > > up the chicken and egg problem.  The broker needs to be
> configured
> > to
> > > > > know
> > > > > > how to talk to you, but you need to configure it before you can
> > talk
> > > to
> > > > > > it.  Using an external secret manager like Vault is one way to
> > solve
> > > > > this,
> > > > > > but not everyone uses an external secret manager.
> > > > > >
> > > > > > quorum.voters seems like a similar configuration key.  In the
> > current
> > > > > KIP,
> > > > > > this is only read if there is no other configuration specifying
> the
> > > > > quorum
> > > > > > voter set.  If we had a kafka.mkfs command, we wouldn't need this
> > key
> > > > > > because we could assume that there was always quorum information
> > > stored
> > > > > > locally.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > >
> > > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > > > > > Hi All,
> > > > > > >
> > > > > > > I'd like to start a discussion on KIP-595:
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > > .
> > > > > > > This proposal specifies a Raft protocol to ultimately replace
> > > > Zookeeper
> > > > > > > as
> > > > > > > documented in KIP-500. Please take a look and share your
> > thoughts.
> > > > > > >
> > > > > > > A few minor notes to set the stage a little bit:
> > > > > > >
> > > > > > > - This KIP does not specify the structure of the messages used
> to
> > > > > > represent
> > > > > > > metadata in Kafka, nor does it specify the internal API that
> will
> > > be
> > > > > used
> > > > > > > by the controller. Expect these to come in later proposals.
> Here
> > we
> > > > are
> > > > > > > primarily concerned with the replication protocol and basic
> > > > operational
> > > > > > > mechanics.
> > > > > > > - We expect many details to change as we get closer to
> > integration
> > > > with
> > > > > > > the controller. Any changes we make will be made either as
> > > amendments
> > > > > to
> > > > > > > this KIP or, in the case of larger changes, as new proposals.
> > > > > > > - We have a prototype implementation which I will put online
> > within
> > > > the
> > > > > > > next week which may help in understanding some details. It has
> > > > > diverged a
> > > > > > > little bit from our proposal, so I am taking a little time to
> > bring
> > > > it
> > > > > in
> > > > > > > line. I'll post an update to this thread when it is available
> for
> > > > > review.
> > > > > > >
> > > > > > > Finally, I want to mention that this proposal was drafted by
> > > myself,
> > > > > > Boyang
> > > > > > > Chen, and Guozhang Wang.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Jason
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Leonard Ge
> > > > > Software Engineer Intern - Confluent
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jun Rao <ju...@confluent.io>.
Hi, Guozhang,

Thanks for the reply. A few more replies inlined below.

On Sun, May 3, 2020 at 6:33 PM Guozhang Wang <wa...@gmail.com> wrote:

> Hello Jun,
>
> Thanks for your comments! I'm replying inline below:
>
> On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:
>
> > 101. Bootstrapping related issues.
> > 101.1 Currently, we support auto broker id generation. Is this supported
> > for bootstrap brokers?
> >
>
> The vote ids would just be the broker ids. "bootstrap.servers" would be
> similar to what client configs have today, where "quorum.voters" would be
> pre-defined config values.
>
>
My question was on the auto generated broker id. Currently, the broker can
choose to have its broker Id auto generated. The generation is done through
ZK to guarantee uniqueness. Without ZK, it's not clear how the broker id is
auto generated. "quorum.voters" also can't be set statically if broker ids
are auto generated.


> > 102. Log compaction. One weak spot of log compaction is for the consumer
> to
> > deal with deletes. When a key is deleted, it's retained as a tombstone
> > first and then physically removed. If a client misses the tombstone
> > (because it's physically removed), it may not be able to update its
> > metadata properly. The way we solve this in Kafka is based on a
> > configuration (log.cleaner.delete.retention.ms) and we expect a consumer
> > having seen an old key to finish reading the deletion tombstone within
> that
> > time. There is no strong guarantee for that since a broker could be down
> > for a long time. It would be better if we can have a more reliable way of
> > dealing with deletes.
> >
>
> We propose to capture this in the "FirstDirtyOffset" field of the quorum
> record fetch response: the offset is the maximum offset that log compaction
> has reached up to. If the follower has fetched beyond this offset it means
> itself is safe hence it has seen all records up to that offset. On getting
> the response, the follower can then decide if its end offset actually below
> that dirty offset (and hence may miss some tombstones). If that's the case:
>
> 1) Naively, it could re-bootstrap metadata log from the very beginning to
> catch up.
> 2) During that time, it would refrain itself from answering MetadataRequest
> from any clients.
>
>
I am not sure that the "FirstDirtyOffset" field fully addresses the issue.
Currently, the deletion tombstone is not removed immediately after a round
of cleaning. It's removed after a delay in a subsequent round of cleaning.
Consider an example where a key insertion is at offset 200 and a deletion
tombstone of the key is at 400. Initially, FirstDirtyOffset is at 300. A
follower/observer fetches from offset 0  and fetches the key at offset 200.
A few rounds of cleaning happen. FirstDirtyOffset is at 500 and the
tombstone at 400 is physically removed. The follower/observer continues the
fetch, but misses offset 400. It catches all the way to FirstDirtyOffset
and declares its metadata as ready. However, its metadata could be stale
since it actually misses the deletion of the key.


> > 105. Quorum State: In addition to VotedId, do we need the epoch
> > corresponding to VotedId? Over time, the same broker Id could be voted in
> > different generations with different epoch.
> >
> >
> Hmm, this is a good point. Originally I think the "LeaderEpoch" field in
> that file is corresponding to the "latest known leader epoch", not the
> "current leader epoch". For example, if the current epoch is N, and then a
> vote-request with epoch N+1 is received and the voter granted the vote for
> it, then it means for this voter it knows the "latest epoch" is N + 1
> although it is unknown if that sending candidate will indeed become the new
> leader (which would only be notified via begin-quorum request). However,
> when persisting the quorum state, we would encode leader-epoch to N+1,
> while the leaderId to be the older leader.
>
> But now thinking about this a bit more, I feel we should use two separate
> epochs, one for the "lates known" and one for the "current" to pair with
> the leaderId. I will update the wiki page.
>
>
Hmm, it's kind of weird to bump up the leader epoch before the new leader
is actually elected, right.


> > 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to indicate
> > that the follower has fetched from an invalid offset and should truncate
> to
> > the offset/epoch indicated in the response." Observers can't truncate
> their
> > logs. What should they do with OFFSET_OUT_OF_RANGE?
> >
> >
> I'm not sure if I understand your question? Observers should still be able
> to truncate their logs as well.
>
>
Hmm, I thought only the quorum nodes have local logs and observers don't?

> 107. "The leader will continue sending BeginQuorumEpoch to each known
> voter
> > until it has received its endorsement." If a voter is down for a long
> time,
> > sending BeginQuorumEpoch seems to add unnecessary overhead. Similarly,
> if a
> > follower stops sending FetchQuorumRecords, does the leader keep sending
> > BeginQuorumEpoch?
> >
>
> Regarding BeginQuorumEpoch: that is a good point. The begin-quorum-epoch
> request is for voters to quickly get the new leader information; however
> even if they do not get them they can still eventually learn about that
> from others via gossiping FindQuorum. I think we can adjust the logic to
> e.g. exponential back-off or with a limited num.retries.
>
> Regarding FetchQuorumRecords: if the follower sends FetchQuorumRecords
> already, it means that follower already knows that the broker is the
> leader, and hence we can stop retrying BeginQuorumEpoch; however it is
> possible that after a follower sends FetchQuorumRecords already, suddenly
> it stops send it (possibly because it learned about a higher epoch leader),
> and hence this broker may be a "zombie" leader and we propose to use the
> fetch.timeout to let the leader to try to verify if it has already been
> stale.
>
>
It just seems that we should handle these two cases in a consistent way?

Thanks,

Jun

>
> >
> > Jun
> >
> > On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Hello Leonard,
> > >
> > > Thanks for your comments, I'm relying in line below:
> > >
> > > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <wg...@confluent.io>
> > > wrote:
> > >
> > > > Hi Kafka developers,
> > > >
> > > > It's great to see this proposal and it took me some time to finish
> > > reading
> > > > it.
> > > >
> > > > And I have the following questions about the Proposal:
> > > >
> > > >    - How do we plan to test this design to ensure its correctness? Or
> > > more
> > > >    broadly, how do we ensure that our new ‘pull’ based model is
> > > functional
> > > > and
> > > >    correct given that it is different from the original RAFT
> > > implementation
> > > >    which has formal proof of correctness?
> > > >
> > >
> > > We have two planned verifications on the correctness and liveness of
> the
> > > design. One is via model verification (TLA+)
> > > https://github.com/guozhangwang/kafka-specification
> > >
> > > Another is via the concurrent simulation tests
> > >
> > >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> > >
> > >    - Have we considered any sensible defaults for the configuration,
> i.e.
> > > >    all the election timeout, fetch time out, etc.? Or we want to
> leave
> > > > this to
> > > >    a later stage when we do the performance testing, etc.
> > > >
> > >
> > > This is a good question, the reason we did not set any default values
> for
> > > the timeout configurations is that we think it may take some
> benchmarking
> > > experiments to get these defaults right. Some high-level principles to
> > > consider: 1) the fetch.timeout should be around the same scale with zk
> > > session timeout, which is now 18 seconds by default -- in practice
> we've
> > > seen unstable networks having more than 10 secs of transient
> > connectivity,
> > > 2) the election.timeout, however, should be smaller than the fetch
> > timeout
> > > as is also suggested as a practical optimization in literature:
> > > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> > >
> > > Some more discussions can be found here:
> > > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> > >
> > >
> > > >    - Have we considered piggybacking `BeginQuorumEpoch` with the `
> > > >    FetchQuorumRecords`? I might be missing something obvious but I am
> > > just
> > > >    wondering why don’t we just use the `FindQuorum` and
> > > > `FetchQuorumRecords`
> > > >    APIs and remove the `BeginQuorumEpoch` API?
> > > >
> > >
> > > Note that Begin/EndQuorumEpoch is sent from leader -> other voter
> > > followers, while FindQuorum / Fetch are sent from follower to leader.
> > > Arguably one can eventually realize the new leader and epoch via
> > gossiping
> > > FindQuorum, but that could in practice require a long delay. Having a
> > > leader -> other voters request helps the new leader epoch to be
> > propagated
> > > faster under a pull model.
> > >
> > >
> > > >    - And about the `FetchQuorumRecords` response schema, in the
> > `Records`
> > > >    field of the response, is it just one record or all the records
> > > starting
> > > >    from the FetchOffset? It seems a lot more efficient if we sent all
> > the
> > > >    records during the bootstrapping of the brokers.
> > > >
> > >
> > > Yes the fetching is batched: FetchOffset is just the starting offset of
> > the
> > > batch of records.
> > >
> > >
> > > >    - Regarding the disruptive broker issues, does our pull based
> model
> > > >    suffer from it? If so, have we considered the Pre-Vote stage? If
> > not,
> > > > why?
> > > >
> > > >
> > > The disruptive broker is stated in the original Raft paper which is the
> > > result of the push model design. Our analysis showed that with the pull
> > > model it is no longer an issue.
> > >
> > >
> > > > Thanks a lot for putting this up, and I hope that my questions can be
> > of
> > > > some value to make this KIP better.
> > > >
> > > > Hope to hear from you soon!
> > > >
> > > > Best wishes,
> > > > Leonard
> > > >
> > > >
> > > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org>
> > wrote:
> > > >
> > > > > Hi Jason,
> > > > >
> > > > > It's amazing to see this coming together :)
> > > > >
> > > > > I haven't had a chance to read in detail, but I read the outline
> and
> > a
> > > > few
> > > > > things jumped out at me.
> > > > >
> > > > > First, for every epoch that is 32 bits rather than 64, I sort of
> > wonder
> > > > if
> > > > > that's a good long-term choice.  I keep reading about stuff like
> > this:
> > > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .  Obviously,
> > > that
> > > > > JIRA is about zxid, which increments much faster than we expect
> these
> > > > > leader epochs to, but it would still be good to see some rough
> > > > calculations
> > > > > about how long 32 bits (or really, 31 bits) will last us in the
> cases
> > > > where
> > > > > we're using it, and what the space savings we're getting really is.
> > It
> > > > > seems like in most cases the tradeoff may not be worth it?
> > > > >
> > > > > Another thing I've been thinking about is how we do
> bootstrapping.  I
> > > > > would prefer to be in a world where formatting a new Kafka node
> was a
> > > > first
> > > > > class operation explicitly initiated by the admin, rather than
> > > something
> > > > > that happened implicitly when you started up the broker and things
> > > > "looked
> > > > > blank."
> > > > >
> > > > > The first problem is that things can "look blank" accidentally if
> the
> > > > > storage system is having a bad day.  Clearly in the non-Raft world,
> > > this
> > > > > leads to data loss if the broker that is (re)started this way was
> the
> > > > > leader for some partitions.
> > > > >
> > > > > The second problem is that we have a bit of a chicken and egg
> problem
> > > > with
> > > > > certain configuration keys.  For example, maybe you want to
> configure
> > > > some
> > > > > connection security settings in your cluster, but you don't want
> them
> > > to
> > > > > ever be stored in a plaintext config file.  (For example, SCRAM
> > > > passwords,
> > > > > etc.)  You could use a broker API to set the configuration, but
> that
> > > > brings
> > > > > up the chicken and egg problem.  The broker needs to be configured
> to
> > > > know
> > > > > how to talk to you, but you need to configure it before you can
> talk
> > to
> > > > > it.  Using an external secret manager like Vault is one way to
> solve
> > > > this,
> > > > > but not everyone uses an external secret manager.
> > > > >
> > > > > quorum.voters seems like a similar configuration key.  In the
> current
> > > > KIP,
> > > > > this is only read if there is no other configuration specifying the
> > > > quorum
> > > > > voter set.  If we had a kafka.mkfs command, we wouldn't need this
> key
> > > > > because we could assume that there was always quorum information
> > stored
> > > > > locally.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > > > > Hi All,
> > > > > >
> > > > > > I'd like to start a discussion on KIP-595:
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > > .
> > > > > > This proposal specifies a Raft protocol to ultimately replace
> > > Zookeeper
> > > > > > as
> > > > > > documented in KIP-500. Please take a look and share your
> thoughts.
> > > > > >
> > > > > > A few minor notes to set the stage a little bit:
> > > > > >
> > > > > > - This KIP does not specify the structure of the messages used to
> > > > > represent
> > > > > > metadata in Kafka, nor does it specify the internal API that will
> > be
> > > > used
> > > > > > by the controller. Expect these to come in later proposals. Here
> we
> > > are
> > > > > > primarily concerned with the replication protocol and basic
> > > operational
> > > > > > mechanics.
> > > > > > - We expect many details to change as we get closer to
> integration
> > > with
> > > > > > the controller. Any changes we make will be made either as
> > amendments
> > > > to
> > > > > > this KIP or, in the case of larger changes, as new proposals.
> > > > > > - We have a prototype implementation which I will put online
> within
> > > the
> > > > > > next week which may help in understanding some details. It has
> > > > diverged a
> > > > > > little bit from our proposal, so I am taking a little time to
> bring
> > > it
> > > > in
> > > > > > line. I'll post an update to this thread when it is available for
> > > > review.
> > > > > >
> > > > > > Finally, I want to mention that this proposal was drafted by
> > myself,
> > > > > Boyang
> > > > > > Chen, and Guozhang Wang.
> > > > > >
> > > > > > Thanks,
> > > > > > Jason
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > Leonard Ge
> > > > Software Engineer Intern - Confluent
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Jun,

Thanks for your comments! I'm replying inline below:

On Fri, May 1, 2020 at 12:36 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Jason,
>
> Thanks for the KIP. Great writeup. A few comments below.
>
> 100. Do we need AlterQuorum in the first version? Quorum changes are rare
> and the implementation is involved. ZK doesn't have that until 4 years
> after the initial version. Dropping that in the first version could speed
> up this KIP.
>

Yes, the first version of the implementation does not necessarily need
quorum re-configuration. In fact, in our current on-going work we did not
put it as the top priorities to complete before we push out the first
working prototype --- the quorum members can just be static “quorum.voters”
config.

However, we still want to include the design of quorum re-configuration in
this KIP for discussion, so that we are confident that when adding the
dynamic reconfiguration feature it is well-aligned with the protocol we've
proposed and implemented, and we would not need to rework a lot of the
implementation in our first version.


> 101. Bootstrapping related issues.
> 101.1 Currently, we support auto broker id generation. Is this supported
> for bootstrap brokers?
>

The vote ids would just be the broker ids. "bootstrap.servers" would be
similar to what client configs have today, where "quorum.voters" would be
pre-defined config values.


> 101.2 As Colin mentioned, sometimes we may need to load the security
> credentials to be broker before it can be connected to. Could you provide a
> bit more detail on how this will work?
>
>
This is a good question.. Either the credentials are stored in a remote
source or in local JAAS file, I think we need to load it before the broker
trying to find out the quorum.



> 102. Log compaction. One weak spot of log compaction is for the consumer to
> deal with deletes. When a key is deleted, it's retained as a tombstone
> first and then physically removed. If a client misses the tombstone
> (because it's physically removed), it may not be able to update its
> metadata properly. The way we solve this in Kafka is based on a
> configuration (log.cleaner.delete.retention.ms) and we expect a consumer
> having seen an old key to finish reading the deletion tombstone within that
> time. There is no strong guarantee for that since a broker could be down
> for a long time. It would be better if we can have a more reliable way of
> dealing with deletes.
>

We propose to capture this in the "FirstDirtyOffset" field of the quorum
record fetch response: the offset is the maximum offset that log compaction
has reached up to. If the follower has fetched beyond this offset it means
itself is safe hence it has seen all records up to that offset. On getting
the response, the follower can then decide if its end offset actually below
that dirty offset (and hence may miss some tombstones). If that's the case:

1) Naively, it could re-bootstrap metadata log from the very beginning to
catch up.
2) During that time, it would refrain itself from answering MetadataRequest
from any clients.


> 103. For the newly introduced configurations related to timeouts, could we
> describe the defaults?
>

We are still discussing about the default values, and I think some
benchmarking experiment would be needed. At the moment just based on the
literature, our general thinking are:

1) the fetch.timeout should be around the same scale with zk session
timeout, which is now 18 seconds by default -- in practice we've seen
unstable networks having more than 10 secs of transient connectivity,
2) the election.timeout, however, should be smaller than the fetch timeout
as is also suggested as a practical optimization in literature:
https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf


> 104. "This proposal requires a persistent log as well as a separate file to
> maintain the current quorum state". In JBOD, are the quorum log and quorum
> state file kept together on the same disk?
>
>
I think for correctness there's no requirement if they have to be either
both intact or corrupted: the quorum-state is the source of truth for the
current quorum state, especially for the current candidate (if any) that
this voter has voted to. If it is missing, the broker would just rely on
the gossiping FindQuorum to refresh its knowledge. The quorum log on the
other hand, is just storing the metadata updates (including the quorum
changes, for reconfiguration purposes) for Kafka. So it is okay if either
of them is corrupted while the other is intact.


> 105. Quorum State: In addition to VotedId, do we need the epoch
> corresponding to VotedId? Over time, the same broker Id could be voted in
> different generations with different epoch.
>
>
Hmm, this is a good point. Originally I think the "LeaderEpoch" field in
that file is corresponding to the "latest known leader epoch", not the
"current leader epoch". For example, if the current epoch is N, and then a
vote-request with epoch N+1 is received and the voter granted the vote for
it, then it means for this voter it knows the "latest epoch" is N + 1
although it is unknown if that sending candidate will indeed become the new
leader (which would only be notified via begin-quorum request). However,
when persisting the quorum state, we would encode leader-epoch to N+1,
while the leaderId to be the older leader.

But now thinking about this a bit more, I feel we should use two separate
epochs, one for the "lates known" and one for the "current" to pair with
the leaderId. I will update the wiki page.


> 106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to indicate
> that the follower has fetched from an invalid offset and should truncate to
> the offset/epoch indicated in the response." Observers can't truncate their
> logs. What should they do with OFFSET_OUT_OF_RANGE?
>
>
I'm not sure if I understand your question? Observers should still be able
to truncate their logs as well.


> 107. "The leader will continue sending BeginQuorumEpoch to each known voter
> until it has received its endorsement." If a voter is down for a long time,
> sending BeginQuorumEpoch seems to add unnecessary overhead. Similarly, if a
> follower stops sending FetchQuorumRecords, does the leader keep sending
> BeginQuorumEpoch?
>

Regarding BeginQuorumEpoch: that is a good point. The begin-quorum-epoch
request is for voters to quickly get the new leader information; however
even if they do not get them they can still eventually learn about that
from others via gossiping FindQuorum. I think we can adjust the logic to
e.g. exponential back-off or with a limited num.retries.

Regarding FetchQuorumRecords: if the follower sends FetchQuorumRecords
already, it means that follower already knows that the broker is the
leader, and hence we can stop retrying BeginQuorumEpoch; however it is
possible that after a follower sends FetchQuorumRecords already, suddenly
it stops send it (possibly because it learned about a higher epoch leader),
and hence this broker may be a "zombie" leader and we propose to use the
fetch.timeout to let the leader to try to verify if it has already been
stale.


>
> 108. Do we store leaderEpoch history and HWM on disk for the quorum log as
> we do for regular logs?
>

Yes we will. I can make that clear in the doc too.


>
> 109. Does FetchQuorumRecordsRequest return only up to HWM for observers?
>
>
Yes, the current logic is that for voters we would return up to log-end
while for observers we would only return up to hwm. This is already
implemented in

https://github.com/confluentinc/kafka/commit/9f52a740aba457dcf27c5babd12d6dbbe31d5e4f

I will update the doc too.



> 110. "Specifically a follower/observer must check for voter assignment
> messages". Do you mean LeaderChangeMessage?
>
>
It is referred to "VoterAssignmentMessage", which is type==2 stored as part
of the metadata log and is to be used for quorum reconfiguration. I will
update the doc.


> 111. FetchQuorumRecords also serves as keep-alive for observers. I am
> wondering if the observer needs a separate EndObserving RPC during a
> controlled shutdown so that the controller can detect the planned failure
> faster than the timeout.
>

We've thought about the special logic for broker start up / broker shutdown
to replace the current ZK ephemeral node mechanism. For broker shutdown,
the current thinking is to still piggy-back on ControlledShutdown request
from broker to controller. Then in addition to partition migration, the
controller would also do extra logic if the shutting down broker is also
part of the quorum (i.e. it is a voter).



>
> Jun
>
> On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hello Leonard,
> >
> > Thanks for your comments, I'm relying in line below:
> >
> > On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <wg...@confluent.io>
> > wrote:
> >
> > > Hi Kafka developers,
> > >
> > > It's great to see this proposal and it took me some time to finish
> > reading
> > > it.
> > >
> > > And I have the following questions about the Proposal:
> > >
> > >    - How do we plan to test this design to ensure its correctness? Or
> > more
> > >    broadly, how do we ensure that our new ‘pull’ based model is
> > functional
> > > and
> > >    correct given that it is different from the original RAFT
> > implementation
> > >    which has formal proof of correctness?
> > >
> >
> > We have two planned verifications on the correctness and liveness of the
> > design. One is via model verification (TLA+)
> > https://github.com/guozhangwang/kafka-specification
> >
> > Another is via the concurrent simulation tests
> >
> >
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
> >
> >    - Have we considered any sensible defaults for the configuration, i.e.
> > >    all the election timeout, fetch time out, etc.? Or we want to leave
> > > this to
> > >    a later stage when we do the performance testing, etc.
> > >
> >
> > This is a good question, the reason we did not set any default values for
> > the timeout configurations is that we think it may take some benchmarking
> > experiments to get these defaults right. Some high-level principles to
> > consider: 1) the fetch.timeout should be around the same scale with zk
> > session timeout, which is now 18 seconds by default -- in practice we've
> > seen unstable networks having more than 10 secs of transient
> connectivity,
> > 2) the election.timeout, however, should be smaller than the fetch
> timeout
> > as is also suggested as a practical optimization in literature:
> > https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
> >
> > Some more discussions can be found here:
> > https://github.com/confluentinc/kafka/pull/301/files#r415420081
> >
> >
> > >    - Have we considered piggybacking `BeginQuorumEpoch` with the `
> > >    FetchQuorumRecords`? I might be missing something obvious but I am
> > just
> > >    wondering why don’t we just use the `FindQuorum` and
> > > `FetchQuorumRecords`
> > >    APIs and remove the `BeginQuorumEpoch` API?
> > >
> >
> > Note that Begin/EndQuorumEpoch is sent from leader -> other voter
> > followers, while FindQuorum / Fetch are sent from follower to leader.
> > Arguably one can eventually realize the new leader and epoch via
> gossiping
> > FindQuorum, but that could in practice require a long delay. Having a
> > leader -> other voters request helps the new leader epoch to be
> propagated
> > faster under a pull model.
> >
> >
> > >    - And about the `FetchQuorumRecords` response schema, in the
> `Records`
> > >    field of the response, is it just one record or all the records
> > starting
> > >    from the FetchOffset? It seems a lot more efficient if we sent all
> the
> > >    records during the bootstrapping of the brokers.
> > >
> >
> > Yes the fetching is batched: FetchOffset is just the starting offset of
> the
> > batch of records.
> >
> >
> > >    - Regarding the disruptive broker issues, does our pull based model
> > >    suffer from it? If so, have we considered the Pre-Vote stage? If
> not,
> > > why?
> > >
> > >
> > The disruptive broker is stated in the original Raft paper which is the
> > result of the push model design. Our analysis showed that with the pull
> > model it is no longer an issue.
> >
> >
> > > Thanks a lot for putting this up, and I hope that my questions can be
> of
> > > some value to make this KIP better.
> > >
> > > Hope to hear from you soon!
> > >
> > > Best wishes,
> > > Leonard
> > >
> > >
> > > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org>
> wrote:
> > >
> > > > Hi Jason,
> > > >
> > > > It's amazing to see this coming together :)
> > > >
> > > > I haven't had a chance to read in detail, but I read the outline and
> a
> > > few
> > > > things jumped out at me.
> > > >
> > > > First, for every epoch that is 32 bits rather than 64, I sort of
> wonder
> > > if
> > > > that's a good long-term choice.  I keep reading about stuff like
> this:
> > > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .  Obviously,
> > that
> > > > JIRA is about zxid, which increments much faster than we expect these
> > > > leader epochs to, but it would still be good to see some rough
> > > calculations
> > > > about how long 32 bits (or really, 31 bits) will last us in the cases
> > > where
> > > > we're using it, and what the space savings we're getting really is.
> It
> > > > seems like in most cases the tradeoff may not be worth it?
> > > >
> > > > Another thing I've been thinking about is how we do bootstrapping.  I
> > > > would prefer to be in a world where formatting a new Kafka node was a
> > > first
> > > > class operation explicitly initiated by the admin, rather than
> > something
> > > > that happened implicitly when you started up the broker and things
> > > "looked
> > > > blank."
> > > >
> > > > The first problem is that things can "look blank" accidentally if the
> > > > storage system is having a bad day.  Clearly in the non-Raft world,
> > this
> > > > leads to data loss if the broker that is (re)started this way was the
> > > > leader for some partitions.
> > > >
> > > > The second problem is that we have a bit of a chicken and egg problem
> > > with
> > > > certain configuration keys.  For example, maybe you want to configure
> > > some
> > > > connection security settings in your cluster, but you don't want them
> > to
> > > > ever be stored in a plaintext config file.  (For example, SCRAM
> > > passwords,
> > > > etc.)  You could use a broker API to set the configuration, but that
> > > brings
> > > > up the chicken and egg problem.  The broker needs to be configured to
> > > know
> > > > how to talk to you, but you need to configure it before you can talk
> to
> > > > it.  Using an external secret manager like Vault is one way to solve
> > > this,
> > > > but not everyone uses an external secret manager.
> > > >
> > > > quorum.voters seems like a similar configuration key.  In the current
> > > KIP,
> > > > this is only read if there is no other configuration specifying the
> > > quorum
> > > > voter set.  If we had a kafka.mkfs command, we wouldn't need this key
> > > > because we could assume that there was always quorum information
> stored
> > > > locally.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > > > Hi All,
> > > > >
> > > > > I'd like to start a discussion on KIP-595:
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > > .
> > > > > This proposal specifies a Raft protocol to ultimately replace
> > Zookeeper
> > > > > as
> > > > > documented in KIP-500. Please take a look and share your thoughts.
> > > > >
> > > > > A few minor notes to set the stage a little bit:
> > > > >
> > > > > - This KIP does not specify the structure of the messages used to
> > > > represent
> > > > > metadata in Kafka, nor does it specify the internal API that will
> be
> > > used
> > > > > by the controller. Expect these to come in later proposals. Here we
> > are
> > > > > primarily concerned with the replication protocol and basic
> > operational
> > > > > mechanics.
> > > > > - We expect many details to change as we get closer to integration
> > with
> > > > > the controller. Any changes we make will be made either as
> amendments
> > > to
> > > > > this KIP or, in the case of larger changes, as new proposals.
> > > > > - We have a prototype implementation which I will put online within
> > the
> > > > > next week which may help in understanding some details. It has
> > > diverged a
> > > > > little bit from our proposal, so I am taking a little time to bring
> > it
> > > in
> > > > > line. I'll post an update to this thread when it is available for
> > > review.
> > > > >
> > > > > Finally, I want to mention that this proposal was drafted by
> myself,
> > > > Boyang
> > > > > Chen, and Guozhang Wang.
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > >
> > >
> > >
> > > --
> > > Leonard Ge
> > > Software Engineer Intern - Confluent
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Jun Rao <ju...@confluent.io>.
Hi, Jason,

Thanks for the KIP. Great writeup. A few comments below.

100. Do we need AlterQuorum in the first version? Quorum changes are rare
and the implementation is involved. ZK doesn't have that until 4 years
after the initial version. Dropping that in the first version could speed
up this KIP.

101. Bootstrapping related issues.
101.1 Currently, we support auto broker id generation. Is this supported
for bootstrap brokers?
101.2 As Colin mentioned, sometimes we may need to load the security
credentials to be broker before it can be connected to. Could you provide a
bit more detail on how this will work?

102. Log compaction. One weak spot of log compaction is for the consumer to
deal with deletes. When a key is deleted, it's retained as a tombstone
first and then physically removed. If a client misses the tombstone
(because it's physically removed), it may not be able to update its
metadata properly. The way we solve this in Kafka is based on a
configuration (log.cleaner.delete.retention.ms) and we expect a consumer
having seen an old key to finish reading the deletion tombstone within that
time. There is no strong guarantee for that since a broker could be down
for a long time. It would be better if we can have a more reliable way of
dealing with deletes.

103. For the newly introduced configurations related to timeouts, could we
describe the defaults?

104. "This proposal requires a persistent log as well as a separate file to
maintain the current quorum state". In JBOD, are the quorum log and quorum
state file kept together on the same disk?

105. Quorum State: In addition to VotedId, do we need the epoch
corresponding to VotedId? Over time, the same broker Id could be voted in
different generations with different epoch.

106. "OFFSET_OUT_OF_RANGE: Used in the FetchQuorumRecords API to indicate
that the follower has fetched from an invalid offset and should truncate to
the offset/epoch indicated in the response." Observers can't truncate their
logs. What should they do with OFFSET_OUT_OF_RANGE?

107. "The leader will continue sending BeginQuorumEpoch to each known voter
until it has received its endorsement." If a voter is down for a long time,
sending BeginQuorumEpoch seems to add unnecessary overhead. Similarly, if a
follower stops sending FetchQuorumRecords, does the leader keep sending
BeginQuorumEpoch?

108. Do we store leaderEpoch history and HWM on disk for the quorum log as
we do for regular logs?

109. Does FetchQuorumRecordsRequest return only up to HWM for observers?

110. "Specifically a follower/observer must check for voter assignment
messages". Do you mean LeaderChangeMessage?

111. FetchQuorumRecords also serves as keep-alive for observers. I am
wondering if the observer needs a separate EndObserving RPC during a
controlled shutdown so that the controller can detect the planned failure
faster than the timeout.

Jun

On Wed, Apr 29, 2020 at 8:56 PM Guozhang Wang <wa...@gmail.com> wrote:

> Hello Leonard,
>
> Thanks for your comments, I'm relying in line below:
>
> On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <wg...@confluent.io>
> wrote:
>
> > Hi Kafka developers,
> >
> > It's great to see this proposal and it took me some time to finish
> reading
> > it.
> >
> > And I have the following questions about the Proposal:
> >
> >    - How do we plan to test this design to ensure its correctness? Or
> more
> >    broadly, how do we ensure that our new ‘pull’ based model is
> functional
> > and
> >    correct given that it is different from the original RAFT
> implementation
> >    which has formal proof of correctness?
> >
>
> We have two planned verifications on the correctness and liveness of the
> design. One is via model verification (TLA+)
> https://github.com/guozhangwang/kafka-specification
>
> Another is via the concurrent simulation tests
>
> https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91
>
>    - Have we considered any sensible defaults for the configuration, i.e.
> >    all the election timeout, fetch time out, etc.? Or we want to leave
> > this to
> >    a later stage when we do the performance testing, etc.
> >
>
> This is a good question, the reason we did not set any default values for
> the timeout configurations is that we think it may take some benchmarking
> experiments to get these defaults right. Some high-level principles to
> consider: 1) the fetch.timeout should be around the same scale with zk
> session timeout, which is now 18 seconds by default -- in practice we've
> seen unstable networks having more than 10 secs of transient connectivity,
> 2) the election.timeout, however, should be smaller than the fetch timeout
> as is also suggested as a practical optimization in literature:
> https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf
>
> Some more discussions can be found here:
> https://github.com/confluentinc/kafka/pull/301/files#r415420081
>
>
> >    - Have we considered piggybacking `BeginQuorumEpoch` with the `
> >    FetchQuorumRecords`? I might be missing something obvious but I am
> just
> >    wondering why don’t we just use the `FindQuorum` and
> > `FetchQuorumRecords`
> >    APIs and remove the `BeginQuorumEpoch` API?
> >
>
> Note that Begin/EndQuorumEpoch is sent from leader -> other voter
> followers, while FindQuorum / Fetch are sent from follower to leader.
> Arguably one can eventually realize the new leader and epoch via gossiping
> FindQuorum, but that could in practice require a long delay. Having a
> leader -> other voters request helps the new leader epoch to be propagated
> faster under a pull model.
>
>
> >    - And about the `FetchQuorumRecords` response schema, in the `Records`
> >    field of the response, is it just one record or all the records
> starting
> >    from the FetchOffset? It seems a lot more efficient if we sent all the
> >    records during the bootstrapping of the brokers.
> >
>
> Yes the fetching is batched: FetchOffset is just the starting offset of the
> batch of records.
>
>
> >    - Regarding the disruptive broker issues, does our pull based model
> >    suffer from it? If so, have we considered the Pre-Vote stage? If not,
> > why?
> >
> >
> The disruptive broker is stated in the original Raft paper which is the
> result of the push model design. Our analysis showed that with the pull
> model it is no longer an issue.
>
>
> > Thanks a lot for putting this up, and I hope that my questions can be of
> > some value to make this KIP better.
> >
> > Hope to hear from you soon!
> >
> > Best wishes,
> > Leonard
> >
> >
> > On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi Jason,
> > >
> > > It's amazing to see this coming together :)
> > >
> > > I haven't had a chance to read in detail, but I read the outline and a
> > few
> > > things jumped out at me.
> > >
> > > First, for every epoch that is 32 bits rather than 64, I sort of wonder
> > if
> > > that's a good long-term choice.  I keep reading about stuff like this:
> > > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .  Obviously,
> that
> > > JIRA is about zxid, which increments much faster than we expect these
> > > leader epochs to, but it would still be good to see some rough
> > calculations
> > > about how long 32 bits (or really, 31 bits) will last us in the cases
> > where
> > > we're using it, and what the space savings we're getting really is.  It
> > > seems like in most cases the tradeoff may not be worth it?
> > >
> > > Another thing I've been thinking about is how we do bootstrapping.  I
> > > would prefer to be in a world where formatting a new Kafka node was a
> > first
> > > class operation explicitly initiated by the admin, rather than
> something
> > > that happened implicitly when you started up the broker and things
> > "looked
> > > blank."
> > >
> > > The first problem is that things can "look blank" accidentally if the
> > > storage system is having a bad day.  Clearly in the non-Raft world,
> this
> > > leads to data loss if the broker that is (re)started this way was the
> > > leader for some partitions.
> > >
> > > The second problem is that we have a bit of a chicken and egg problem
> > with
> > > certain configuration keys.  For example, maybe you want to configure
> > some
> > > connection security settings in your cluster, but you don't want them
> to
> > > ever be stored in a plaintext config file.  (For example, SCRAM
> > passwords,
> > > etc.)  You could use a broker API to set the configuration, but that
> > brings
> > > up the chicken and egg problem.  The broker needs to be configured to
> > know
> > > how to talk to you, but you need to configure it before you can talk to
> > > it.  Using an external secret manager like Vault is one way to solve
> > this,
> > > but not everyone uses an external secret manager.
> > >
> > > quorum.voters seems like a similar configuration key.  In the current
> > KIP,
> > > this is only read if there is no other configuration specifying the
> > quorum
> > > voter set.  If we had a kafka.mkfs command, we wouldn't need this key
> > > because we could assume that there was always quorum information stored
> > > locally.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > > Hi All,
> > > >
> > > > I'd like to start a discussion on KIP-595:
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > > .
> > > > This proposal specifies a Raft protocol to ultimately replace
> Zookeeper
> > > > as
> > > > documented in KIP-500. Please take a look and share your thoughts.
> > > >
> > > > A few minor notes to set the stage a little bit:
> > > >
> > > > - This KIP does not specify the structure of the messages used to
> > > represent
> > > > metadata in Kafka, nor does it specify the internal API that will be
> > used
> > > > by the controller. Expect these to come in later proposals. Here we
> are
> > > > primarily concerned with the replication protocol and basic
> operational
> > > > mechanics.
> > > > - We expect many details to change as we get closer to integration
> with
> > > > the controller. Any changes we make will be made either as amendments
> > to
> > > > this KIP or, in the case of larger changes, as new proposals.
> > > > - We have a prototype implementation which I will put online within
> the
> > > > next week which may help in understanding some details. It has
> > diverged a
> > > > little bit from our proposal, so I am taking a little time to bring
> it
> > in
> > > > line. I'll post an update to this thread when it is available for
> > review.
> > > >
> > > > Finally, I want to mention that this proposal was drafted by myself,
> > > Boyang
> > > > Chen, and Guozhang Wang.
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > >
> >
> >
> > --
> > Leonard Ge
> > Software Engineer Intern - Confluent
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Leonard,

Thanks for your comments, I'm relying in line below:

On Wed, Apr 29, 2020 at 1:58 AM Wang (Leonard) Ge <wg...@confluent.io> wrote:

> Hi Kafka developers,
>
> It's great to see this proposal and it took me some time to finish reading
> it.
>
> And I have the following questions about the Proposal:
>
>    - How do we plan to test this design to ensure its correctness? Or more
>    broadly, how do we ensure that our new ‘pull’ based model is functional
> and
>    correct given that it is different from the original RAFT implementation
>    which has formal proof of correctness?
>

We have two planned verifications on the correctness and liveness of the
design. One is via model verification (TLA+)
https://github.com/guozhangwang/kafka-specification

Another is via the concurrent simulation tests
https://github.com/confluentinc/kafka/commit/5c0c054597d2d9f458cad0cad846b0671efa2d91

   - Have we considered any sensible defaults for the configuration, i.e.
>    all the election timeout, fetch time out, etc.? Or we want to leave
> this to
>    a later stage when we do the performance testing, etc.
>

This is a good question, the reason we did not set any default values for
the timeout configurations is that we think it may take some benchmarking
experiments to get these defaults right. Some high-level principles to
consider: 1) the fetch.timeout should be around the same scale with zk
session timeout, which is now 18 seconds by default -- in practice we've
seen unstable networks having more than 10 secs of transient connectivity,
2) the election.timeout, however, should be smaller than the fetch timeout
as is also suggested as a practical optimization in literature:
https://www.cl.cam.ac.uk/~ms705/pub/papers/2015-osr-raft.pdf

Some more discussions can be found here:
https://github.com/confluentinc/kafka/pull/301/files#r415420081


>    - Have we considered piggybacking `BeginQuorumEpoch` with the `
>    FetchQuorumRecords`? I might be missing something obvious but I am just
>    wondering why don’t we just use the `FindQuorum` and
> `FetchQuorumRecords`
>    APIs and remove the `BeginQuorumEpoch` API?
>

Note that Begin/EndQuorumEpoch is sent from leader -> other voter
followers, while FindQuorum / Fetch are sent from follower to leader.
Arguably one can eventually realize the new leader and epoch via gossiping
FindQuorum, but that could in practice require a long delay. Having a
leader -> other voters request helps the new leader epoch to be propagated
faster under a pull model.


>    - And about the `FetchQuorumRecords` response schema, in the `Records`
>    field of the response, is it just one record or all the records starting
>    from the FetchOffset? It seems a lot more efficient if we sent all the
>    records during the bootstrapping of the brokers.
>

Yes the fetching is batched: FetchOffset is just the starting offset of the
batch of records.


>    - Regarding the disruptive broker issues, does our pull based model
>    suffer from it? If so, have we considered the Pre-Vote stage? If not,
> why?
>
>
The disruptive broker is stated in the original Raft paper which is the
result of the push model design. Our analysis showed that with the pull
model it is no longer an issue.


> Thanks a lot for putting this up, and I hope that my questions can be of
> some value to make this KIP better.
>
> Hope to hear from you soon!
>
> Best wishes,
> Leonard
>
>
> On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org> wrote:
>
> > Hi Jason,
> >
> > It's amazing to see this coming together :)
> >
> > I haven't had a chance to read in detail, but I read the outline and a
> few
> > things jumped out at me.
> >
> > First, for every epoch that is 32 bits rather than 64, I sort of wonder
> if
> > that's a good long-term choice.  I keep reading about stuff like this:
> > https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .  Obviously, that
> > JIRA is about zxid, which increments much faster than we expect these
> > leader epochs to, but it would still be good to see some rough
> calculations
> > about how long 32 bits (or really, 31 bits) will last us in the cases
> where
> > we're using it, and what the space savings we're getting really is.  It
> > seems like in most cases the tradeoff may not be worth it?
> >
> > Another thing I've been thinking about is how we do bootstrapping.  I
> > would prefer to be in a world where formatting a new Kafka node was a
> first
> > class operation explicitly initiated by the admin, rather than something
> > that happened implicitly when you started up the broker and things
> "looked
> > blank."
> >
> > The first problem is that things can "look blank" accidentally if the
> > storage system is having a bad day.  Clearly in the non-Raft world, this
> > leads to data loss if the broker that is (re)started this way was the
> > leader for some partitions.
> >
> > The second problem is that we have a bit of a chicken and egg problem
> with
> > certain configuration keys.  For example, maybe you want to configure
> some
> > connection security settings in your cluster, but you don't want them to
> > ever be stored in a plaintext config file.  (For example, SCRAM
> passwords,
> > etc.)  You could use a broker API to set the configuration, but that
> brings
> > up the chicken and egg problem.  The broker needs to be configured to
> know
> > how to talk to you, but you need to configure it before you can talk to
> > it.  Using an external secret manager like Vault is one way to solve
> this,
> > but not everyone uses an external secret manager.
> >
> > quorum.voters seems like a similar configuration key.  In the current
> KIP,
> > this is only read if there is no other configuration specifying the
> quorum
> > voter set.  If we had a kafka.mkfs command, we wouldn't need this key
> > because we could assume that there was always quorum information stored
> > locally.
> >
> > best,
> > Colin
> >
> >
> > On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > > Hi All,
> > >
> > > I'd like to start a discussion on KIP-595:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> > .
> > > This proposal specifies a Raft protocol to ultimately replace Zookeeper
> > > as
> > > documented in KIP-500. Please take a look and share your thoughts.
> > >
> > > A few minor notes to set the stage a little bit:
> > >
> > > - This KIP does not specify the structure of the messages used to
> > represent
> > > metadata in Kafka, nor does it specify the internal API that will be
> used
> > > by the controller. Expect these to come in later proposals. Here we are
> > > primarily concerned with the replication protocol and basic operational
> > > mechanics.
> > > - We expect many details to change as we get closer to integration with
> > > the controller. Any changes we make will be made either as amendments
> to
> > > this KIP or, in the case of larger changes, as new proposals.
> > > - We have a prototype implementation which I will put online within the
> > > next week which may help in understanding some details. It has
> diverged a
> > > little bit from our proposal, so I am taking a little time to bring it
> in
> > > line. I'll post an update to this thread when it is available for
> review.
> > >
> > > Finally, I want to mention that this proposal was drafted by myself,
> > Boyang
> > > Chen, and Guozhang Wang.
> > >
> > > Thanks,
> > > Jason
> > >
> >
>
>
> --
> Leonard Ge
> Software Engineer Intern - Confluent
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by "Wang (Leonard) Ge" <wg...@confluent.io>.
Hi Kafka developers,

It's great to see this proposal and it took me some time to finish reading
it.

And I have the following questions about the Proposal:

   - How do we plan to test this design to ensure its correctness? Or more
   broadly, how do we ensure that our new ‘pull’ based model is functional and
   correct given that it is different from the original RAFT implementation
   which has formal proof of correctness?
   - Have we considered any sensible defaults for the configuration, i.e.
   all the election timeout, fetch time out, etc.? Or we want to leave this to
   a later stage when we do the performance testing, etc.
   - Have we considered piggybacking `BeginQuorumEpoch` with the `
   FetchQuorumRecords`? I might be missing something obvious but I am just
   wondering why don’t we just use the `FindQuorum` and `FetchQuorumRecords`
   APIs and remove the `BeginQuorumEpoch` API?
   - And about the `FetchQuorumRecords` response schema, in the `Records`
   field of the response, is it just one record or all the records starting
   from the FetchOffset? It seems a lot more efficient if we sent all the
   records during the bootstrapping of the brokers.
   - Regarding the disruptive broker issues, does our pull based model
   suffer from it? If so, have we considered the Pre-Vote stage? If not, why?

Thanks a lot for putting this up, and I hope that my questions can be of
some value to make this KIP better.

Hope to hear from you soon!

Best wishes,
Leonard


On Wed, Apr 29, 2020 at 1:46 AM Colin McCabe <cm...@apache.org> wrote:

> Hi Jason,
>
> It's amazing to see this coming together :)
>
> I haven't had a chance to read in detail, but I read the outline and a few
> things jumped out at me.
>
> First, for every epoch that is 32 bits rather than 64, I sort of wonder if
> that's a good long-term choice.  I keep reading about stuff like this:
> https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .  Obviously, that
> JIRA is about zxid, which increments much faster than we expect these
> leader epochs to, but it would still be good to see some rough calculations
> about how long 32 bits (or really, 31 bits) will last us in the cases where
> we're using it, and what the space savings we're getting really is.  It
> seems like in most cases the tradeoff may not be worth it?
>
> Another thing I've been thinking about is how we do bootstrapping.  I
> would prefer to be in a world where formatting a new Kafka node was a first
> class operation explicitly initiated by the admin, rather than something
> that happened implicitly when you started up the broker and things "looked
> blank."
>
> The first problem is that things can "look blank" accidentally if the
> storage system is having a bad day.  Clearly in the non-Raft world, this
> leads to data loss if the broker that is (re)started this way was the
> leader for some partitions.
>
> The second problem is that we have a bit of a chicken and egg problem with
> certain configuration keys.  For example, maybe you want to configure some
> connection security settings in your cluster, but you don't want them to
> ever be stored in a plaintext config file.  (For example, SCRAM passwords,
> etc.)  You could use a broker API to set the configuration, but that brings
> up the chicken and egg problem.  The broker needs to be configured to know
> how to talk to you, but you need to configure it before you can talk to
> it.  Using an external secret manager like Vault is one way to solve this,
> but not everyone uses an external secret manager.
>
> quorum.voters seems like a similar configuration key.  In the current KIP,
> this is only read if there is no other configuration specifying the quorum
> voter set.  If we had a kafka.mkfs command, we wouldn't need this key
> because we could assume that there was always quorum information stored
> locally.
>
> best,
> Colin
>
>
> On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> > Hi All,
> >
> > I'd like to start a discussion on KIP-595:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum
> .
> > This proposal specifies a Raft protocol to ultimately replace Zookeeper
> > as
> > documented in KIP-500. Please take a look and share your thoughts.
> >
> > A few minor notes to set the stage a little bit:
> >
> > - This KIP does not specify the structure of the messages used to
> represent
> > metadata in Kafka, nor does it specify the internal API that will be used
> > by the controller. Expect these to come in later proposals. Here we are
> > primarily concerned with the replication protocol and basic operational
> > mechanics.
> > - We expect many details to change as we get closer to integration with
> > the controller. Any changes we make will be made either as amendments to
> > this KIP or, in the case of larger changes, as new proposals.
> > - We have a prototype implementation which I will put online within the
> > next week which may help in understanding some details. It has diverged a
> > little bit from our proposal, so I am taking a little time to bring it in
> > line. I'll post an update to this thread when it is available for review.
> >
> > Finally, I want to mention that this proposal was drafted by myself,
> Boyang
> > Chen, and Guozhang Wang.
> >
> > Thanks,
> > Jason
> >
>


-- 
Leonard Ge
Software Engineer Intern - Confluent

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
Hi Jason,

It's amazing to see this coming together :)

I haven't had a chance to read in detail, but I read the outline and a few things jumped out at me.

First, for every epoch that is 32 bits rather than 64, I sort of wonder if that's a good long-term choice.  I keep reading about stuff like this: https://issues.apache.org/jira/browse/ZOOKEEPER-1277 .  Obviously, that JIRA is about zxid, which increments much faster than we expect these leader epochs to, but it would still be good to see some rough calculations about how long 32 bits (or really, 31 bits) will last us in the cases where we're using it, and what the space savings we're getting really is.  It seems like in most cases the tradeoff may not be worth it?

Another thing I've been thinking about is how we do bootstrapping.  I would prefer to be in a world where formatting a new Kafka node was a first class operation explicitly initiated by the admin, rather than something that happened implicitly when you started up the broker and things "looked blank."

The first problem is that things can "look blank" accidentally if the storage system is having a bad day.  Clearly in the non-Raft world, this leads to data loss if the broker that is (re)started this way was the leader for some partitions.

The second problem is that we have a bit of a chicken and egg problem with certain configuration keys.  For example, maybe you want to configure some connection security settings in your cluster, but you don't want them to ever be stored in a plaintext config file.  (For example, SCRAM passwords, etc.)  You could use a broker API to set the configuration, but that brings up the chicken and egg problem.  The broker needs to be configured to know how to talk to you, but you need to configure it before you can talk to it.  Using an external secret manager like Vault is one way to solve this, but not everyone uses an external secret manager.

quorum.voters seems like a similar configuration key.  In the current KIP, this is only read if there is no other configuration specifying the quorum voter set.  If we had a kafka.mkfs command, we wouldn't need this key because we could assume that there was always quorum information stored locally.

best,
Colin


On Thu, Apr 16, 2020, at 16:44, Jason Gustafson wrote:
> Hi All,
> 
> I'd like to start a discussion on KIP-595:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum.
> This proposal specifies a Raft protocol to ultimately replace Zookeeper 
> as
> documented in KIP-500. Please take a look and share your thoughts.
> 
> A few minor notes to set the stage a little bit:
> 
> - This KIP does not specify the structure of the messages used to represent
> metadata in Kafka, nor does it specify the internal API that will be used
> by the controller. Expect these to come in later proposals. Here we are
> primarily concerned with the replication protocol and basic operational
> mechanics.
> - We expect many details to change as we get closer to integration with
> the controller. Any changes we make will be made either as amendments to
> this KIP or, in the case of larger changes, as new proposals.
> - We have a prototype implementation which I will put online within the
> next week which may help in understanding some details. It has diverged a
> little bit from our proposal, so I am taking a little time to bring it in
> line. I'll post an update to this thread when it is available for review.
> 
> Finally, I want to mention that this proposal was drafted by myself, Boyang
> Chen, and Guozhang Wang.
> 
> Thanks,
> Jason
>

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

Posted by jianjie feng <au...@gmail.com>.
Hi, Jason
    I'd like to attend to this project and do some development, could you tell me where shall I started?
    Thanks
    Jianjie 
On 2020/04/16 23:44:59, Jason Gustafson <ja...@confluent.io> wrote: 
> Hi All,
> 
> I'd like to start a discussion on KIP-595:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum.
> This proposal specifies a Raft protocol to ultimately replace Zookeeper as
> documented in KIP-500. Please take a look and share your thoughts.
> 
> A few minor notes to set the stage a little bit:
> 
> - This KIP does not specify the structure of the messages used to represent
> metadata in Kafka, nor does it specify the internal API that will be used
> by the controller. Expect these to come in later proposals. Here we are
> primarily concerned with the replication protocol and basic operational
> mechanics.
> - We expect many details to change as we get closer to integration with
> the controller. Any changes we make will be made either as amendments to
> this KIP or, in the case of larger changes, as new proposals.
> - We have a prototype implementation which I will put online within the
> next week which may help in understanding some details. It has diverged a
> little bit from our proposal, so I am taking a little time to bring it in
> line. I'll post an update to this thread when it is available for review.
> 
> Finally, I want to mention that this proposal was drafted by myself, Boyang
> Chen, and Guozhang Wang.
> 
> Thanks,
> Jason
>