You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Colin McCabe <cm...@apache.org> on 2019/08/01 21:04:46 UTC

[DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Hi all,

I've written a KIP about removing ZooKeeper from Kafka.  Please take a look and let me know what you think:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum

cheers,
Colin

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

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

Apache Ratis looks like a very interesting project, but I don't think this is the right use-case for it.  At its heart, Apache Kafka is a system for managing logs.  We should avoid adding a dependency on an external system to manage the logs of Kafka itself, since that is one of Kafka's core functions.

In the past we've successfully used Kafka itself to store metadata about consumer offsets, transactions, and so on.  This is the same kind of thing, except that we need a slightly different replication protocol to avoid the dependency that the existing one has on the controller.

I think that down the road, having support for quorum replication in the Kafka will be useful for regular topics, not just for the metadata partition.  Quorum-based replication has dramatically lower tail latencies than the acks=all configuration that many people use currently.  The tradeoff is that Raft doesn't support redundancy with fewer than 3 replicas.  But that is a tradeoff that is appropriate to make for many applications.

best,
Colin

On Wed, Aug 21, 2019, at 12:19, Ryanne Dolan wrote:
> Colin, have you considered leveraging Apache Ratis (incubating)?
> 
> Ryanne
> 
> On Wed, Aug 21, 2019, 1:28 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > On Wed, Aug 21, 2019, at 06:38, Eno Thereska wrote:
> > > Hi Colin,
> > >
> > > Nice KIP! For such a big change it would be good to add a pointer or
> > > two to related work that provides some sort of soft proof that the
> > > approach taken makes sense. Also such work often builds on other work
> > > and it might be useful to trace its roots. May I recommend adding a
> > > pointer to "Tango: Distributed Data Structures over a Shared Log"
> > > (http://www.cs.cornell.edu/~taozou/sosp13/tangososp.pdf)? There are
> > > other papers that are related (e.g., a more recent one one "The
> > > FuzzyLog: A Partially Ordered Shared Log"
> > > (https://www.usenix.org/system/files/osdi18-lockerman.pdf)).
> > >
> > > Both papers would add to the strength of your motivation.
> > >
> >
> > Hi Eno,
> >
> > Good point.  I added a "references" section on the end and added the Tango
> > paper.  I am not sure we need the FuzzyLog one, though.
> >
> > I also added a link to the Raft paper and one of the papers on HDFS, since
> > I feel like these are very relevant here.
> >
> > best,
> > Colin
> >
> > > Cheers
> > > Eno
> > >
> > > On Wed, Aug 21, 2019 at 12:22 PM Ron Dagostino <rn...@gmail.com>
> > wrote:
> > > >
> > > > Hi Colin.  I like the concept of a "bridge release" for migrating off
> > of
> > > > Zookeeper, but I worry that it may become a bottleneck if people
> > hesitate
> > > > to replace Zookeeper -- they would be unable to adopt newer versions of
> > > > Kafka until taking (what feels to them like) a giant leap.  As an
> > example,
> > > > assuming version 4.0.x of Kafka is the supported bridge release, I
> > would
> > > > not be surprised if uptake of the 4.x release and the time-based
> > releases
> > > > that follow it end up being much slower due to the perceived barrier.
> > > >
> > > > Any perceived barrier could be lowered if the 4.0.x release could
> > > > optionally continue to use Zookeeper -- then the cutover would be two
> > > > incremental steps (move to 4.0.x, then replace Zookeeper while staying
> > on
> > > > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> > > > Zookeeper in one fell swoop).
> > > >
> > > > Regardless of whether what I wrote above has merit or not, I think the
> > KIP
> > > > should be more explicit about what the upgrade constraints actually
> > are.
> > > > Can the bridge release be adopted with Zookeeper remaining in place and
> > > > then cutting over as a second, follow-on step, or must the Controller
> > > > Quorum nodes be started first and the bridge release cannot be used
> > with
> > > > Zookeeper at all?  If the bridge release cannot be used with Zookeeper
> > at
> > > > all, then no version at or beyond the bridge release is available
> > > > unless/until abandoning Zookeeper; if the bridge release can be used
> > with
> > > > Zookeeper, then is it the only version that can be used with
> > Zookeeper, or
> > > > can Zookeeper be kept for additional releases if desired?
> > > >
> > > > Ron
> > > >
> > > > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com>
> > wrote:
> > > >
> > > > > Hi Colin.  The diagram up at the top confused me -- specifically, the
> > > > > lines connecting the controller/active-controller to the brokers.  I
> > had
> > > > > assumed the arrows on those lines represented the direction of data
> > flow,
> > > > > but that is not the case; the arrows actually identify the target of
> > the
> > > > > action, and the non-arrowed end indicates the initiator of the
> > action.  For
> > > > > example, the lines point from the controller to the brokers in the
> > "today"
> > > > > section on the left to show that the controller pushes to the
> > brokers; the
> > > > > lines point from the brokers to the active-controller in the
> > "tomorrow"
> > > > > section on the right to show that the brokers pull from the
> > > > > active-controller.  As I said, this confused me because my gut
> > instinct was
> > > > > to interpret the arrow as indicating the direction of data flow, and
> > when I
> > > > > look at the "tomorrow" picture on the right I initially thought
> > information
> > > > > was moving from the brokers to the active-controller.  Did you
> > consider
> > > > > drawing that picture with the arrows reversed in the "tomorrow" side
> > so
> > > > > that the arrows represent the direction of data flow, and then add
> > the
> > > > > labels "push" on the "today" side and "pull" on the "tomorrow" side
> > to
> > > > > indicate who initiates the data flow?  It occurs to me that this
> > picture
> > > > > may end up being widely distributed, so it might be in everyone's
> > interest
> > > > > to proactively avoid any possible confusion by being more explicit.
> > > > >
> > > > > Minor corrections?
> > > > > <<<In the current world, a broker which can contact ZooKeeper but
> > which
> > > > > is partitioned from the active controller
> > > > > >>>In the current world, a broker which can contact ZooKeeper but
> > which
> > > > > is partitioned from the controller
> > > > >
> > > > > <<<Eventually, the controller will ask the broker to finally go
> > offline
> > > > > >>>Eventually, the active controller will ask the broker to finally
> > go
> > > > > offline
> > > > >
> > > > > <<<New versions of the clients should send these operations directly
> > to
> > > > > the controller
> > > > > >>>New versions of the clients should send these operations directly
> > to
> > > > > the active controller
> > > > >
> > > > > <<<In the post-ZK world, the leader will make an RPC to the
> > controller
> > > > > instead
> > > > > >>>In the post-ZK world, the leader will make an RPC to the active
> > > > > controller instead
> > > > >
> > > > > <<<For example, the brokers may need to forward their requests to the
> > > > > controller.
> > > > > >>>For example, the brokers may need to forward their requests to the
> > > > > active controller.
> > > > >
> > > > > <<<The new controller will monitor ZooKeeper for legacy broker node
> > > > > registrations
> > > > > >>>The new (active) controller will monitor ZooKeeper for legacy
> > broker
> > > > > node registrations
> > > > >
> > > > > Ron
> > > > >
> > > > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > >
> > > > >> Hi all,
> > > > >>
> > > > >> The KIP has been out for a while, so I'm thinking about calling a
> > vote
> > > > >> some time this week.
> > > > >>
> > > > >> best,
> > > > >> Colin
> > > > >>
> > > > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > > >> > > Thanks for the KIP, Colin. This looks great!
> > > > >> > >
> > > > >> > > I really like the idea of separating the Controller and Broker
> > JVMs.
> > > > >> > >
> > > > >> > > As you alluded to above, it might be nice to have a separate
> > > > >> > > broker-registration API to avoid overloading the metadata fetch
> > API.
> > > > >> > >
> > > > >> >
> > > > >> > Hi David,
> > > > >> >
> > > > >> > Thanks for taking a look.
> > > > >> >
> > > > >> > I removed the sentence about MetadataFetch also serving as the
> > broker
> > > > >> > registration API.  I think I agree that we will probably want a
> > > > >> > separate RPC to fill this role.  We will have a follow-on KIP
> > that will
> > > > >> > go into more detail about metadata propagation and registration
> > in the
> > > > >> > post-ZK world.  That KIP will also have a full description of the
> > > > >> > registration RPC, etc.  For now, I think the important part for
> > KIP-500
> > > > >> > is that the broker registers with the controller quorum.  On
> > > > >> > registration, the controller quorum assigns it a new broker epoch,
> > > > >> > which can distinguish successive broker incarnations.
> > > > >> >
> > > > >> > >
> > > > >> > > When a broker gets a metadata delta, will it be a sequence of
> > deltas
> > > > >> since
> > > > >> > > the last update or a cumulative delta since the last update?
> > > > >> > >
> > > > >> >
> > > > >> > It will be a sequence of deltas.  Basically, the broker will be
> > reading
> > > > >> > from the metadata log.
> > > > >> >
> > > > >> > >
> > > > >> > > Will we include any kind of integrity check on the deltas to
> > ensure
> > > > >> the brokers
> > > > >> > > have applied them correctly? Perhaps this will be addressed in
> > one of
> > > > >> the
> > > > >> > > follow-on KIPs.
> > > > >> > >
> > > > >> >
> > > > >> > In general, we will have checksums on the metadata that we
> > fetch.  This
> > > > >> > is similar to how we have checksums on regular data.  Or if the
> > > > >> > question is about catching logic errors in the metadata handling
> > code,
> > > > >> > that sounds more like something that should be caught by test
> > cases.
> > > > >> >
> > > > >> > best,
> > > > >> > Colin
> > > > >> >
> > > > >> >
> > > > >> > > Thanks!
> > > > >> > >
> > > > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cmccabe@apache.org
> > >
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Hi Mickael,
> > > > >> > > >
> > > > >> > > > Thanks for taking a look.
> > > > >> > > >
> > > > >> > > > I don't think we want to support that kind of multi-tenancy
> > at the
> > > > >> > > > controller level.  If the cluster is small enough that we
> > want to
> > > > >> pack the
> > > > >> > > > controller(s) with something else, we could run them
> > alongside the
> > > > >> brokers,
> > > > >> > > > or possibly inside three of the broker JVMs.
> > > > >> > > >
> > > > >> > > > best,
> > > > >> > > > Colin
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > >> > > > > Thank Colin for kickstarting this initiative.
> > > > >> > > > >
> > > > >> > > > > Just one question.
> > > > >> > > > > - A nice feature of Zookeeper is the ability to use chroots
> > and
> > > > >> have
> > > > >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is
> > this
> > > > >> > > > > something we should keep?
> > > > >> > > > >
> > > > >> > > > > Thanks
> > > > >> > > > >
> > > > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> > cmccabe@apache.org>
> > > > >> wrote:
> > > > >> > > > > >
> > > > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > >> > > > > > > Hi Colin,
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks for the KIP.
> > > > >> > > > > > >
> > > > >> > > > > > > Currently ZooKeeper provides a convenient notification
> > > > >> mechanism for
> > > > >> > > > > > > knowing that broker and topic configuration has
> > changed. While
> > > > >> > > > KIP-500 does
> > > > >> > > > > > > suggest that incremental metadata update is expected to
> > come
> > > > >> to
> > > > >> > > > clients
> > > > >> > > > > > > eventually, that would seem to imply that for some
> > number of
> > > > >> > > > releases there
> > > > >> > > > > > > would be no equivalent mechanism for knowing about
> > config
> > > > >> changes.
> > > > >> > > > Is there
> > > > >> > > > > > > any thinking at this point about how a similar
> > notification
> > > > >> might be
> > > > >> > > > > > > provided in the future?
> > > > >> > > > > >
> > > > >> > > > > > We could eventually have some inotify-like mechanism where
> > > > >> clients
> > > > >> > > > could register interest in various types of events and got
> > notified
> > > > >> when
> > > > >> > > > they happened.  Reading the metadata log is conceptually
> > simple.
> > > > >> The main
> > > > >> > > > complexity would be in setting up an API that made sense and
> > that
> > > > >> didn't
> > > > >> > > > unduly constrain future implementations.  We'd have to think
> > > > >> carefully
> > > > >> > > > about what the real use-cases for this were, though.
> > > > >> > > > > >
> > > > >> > > > > > best,
> > > > >> > > > > > Colin
> > > > >> > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks,
> > > > >> > > > > > >
> > > > >> > > > > > > Tom
> > > > >> > > > > > >
> > > > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > > >> > > > viktorsomogyi@gmail.com>
> > > > >> > > > > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hey Colin,
> > > > >> > > > > > > >
> > > > >> > > > > > > > I think this is a long-awaited KIP, thanks for
> > driving it.
> > > > >> I'm
> > > > >> > > > excited to
> > > > >> > > > > > > > see this in Kafka once. I collected my questions (and
> > I
> > > > >> accept the
> > > > >> > > > "TBD"
> > > > >> > > > > > > > answer as they might be a bit deep for this high
> > level :) ).
> > > > >> > > > > > > > 1.) Are there any specific reasons for the Controller
> > just
> > > > >> > > > periodically
> > > > >> > > > > > > > persisting its state on disk periodically instead of
> > > > >> > > > asynchronously with
> > > > >> > > > > > > > every update? Wouldn't less frequent saves increase
> > the
> > > > >> chance for
> > > > >> > > > missing
> > > > >> > > > > > > > a state change if the controller crashes between two
> > saves?
> > > > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from
> > the
> > > > >> follower
> > > > >> > > > > > > > controllers? I assume that followers would have
> > up-to-date
> > > > >> > > > information
> > > > >> > > > > > > > therefore brokers could fetch from there in theory.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks,
> > > > >> > > > > > > > Viktor
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > > >> > > > reluctanthero104@gmail.com>
> > > > >> > > > > > > > wrote:
> > > > >> > > > > > > >
> > > > >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> > > > >> follow-up KIPs
> > > > >> > > > sounds
> > > > >> > > > > > > > like
> > > > >> > > > > > > > > a good idea.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > > > >> ismael@juma.me.uk>
> > > > >> > > > wrote:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > > Hi Boyang,
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > Yes, there will be several KIPs that will discuss
> > the
> > > > >> items you
> > > > >> > > > > > > > describe
> > > > >> > > > > > > > > in
> > > > >> > > > > > > > > > detail. Colin, it may be helpful to make this
> > clear in
> > > > >> the KIP
> > > > >> > > > 500
> > > > >> > > > > > > > > > description.
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > Ismael
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > > >> > > > reluctanthero104@gmail.com
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > > wrote:
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > > Thanks Colin for initiating this important
> > effort!
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > One question I have is whether we have a session
> > > > >> discussing
> > > > >> > > > the
> > > > >> > > > > > > > > > controller
> > > > >> > > > > > > > > > > failover in the new architecture? I know we are
> > using
> > > > >> Raft
> > > > >> > > > protocol
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > > > failover, yet it's still valuable to discuss the
> > > > >> steps new
> > > > >> > > > cluster is
> > > > >> > > > > > > > > > going
> > > > >> > > > > > > > > > > to take to reach the stable stage again, so
> > that we
> > > > >> could
> > > > >> > > > easily
> > > > >> > > > > > > > > measure
> > > > >> > > > > > > > > > > the availability of the metadata servers.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > Another suggestion I have is to write a
> > step-by-step
> > > > >> design
> > > > >> > > > doc like
> > > > >> > > > > > > > > what
> > > > >> > > > > > > > > > > we did in KIP-98
> > > > >> > > > > > > > > > > <
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > >
> > > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >> > > > > > > > > > > >,
> > > > >> > > > > > > > > > > including the new request protocols and how
> > they are
> > > > >> > > > interacting in
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > new
> > > > >> > > > > > > > > > > cluster. For a complicated change like this, an
> > > > >> > > > implementation design
> > > > >> > > > > > > > > doc
> > > > >> > > > > > > > > > > help a lot in the review process, otherwise most
> > > > >> discussions
> > > > >> > > > we have
> > > > >> > > > > > > > > will
> > > > >> > > > > > > > > > > focus on high level and lose important details
> > as we
> > > > >> > > > discover them in
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > > post-agreement phase.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > Boyang
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > > >> > > > cmccabe@apache.org>
> > > > >> > > > > > > > > wrote:
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando
> > Garcia
> > > > >> Sancio
> > > > >> > > > wrote:
> > > > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a
> > few
> > > > >> comments
> > > > >> > > > and
> > > > >> > > > > > > > > questions.
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> > > > >> mentioned the
> > > > >> > > > > > > > LeaderAndIsr
> > > > >> > > > > > > > > > and
> > > > >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates
> > which
> > > > >> the
> > > > >> > > > controller
> > > > >> > > > > > > > > > pushes,
> > > > >> > > > > > > > > > > > such
> > > > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata
> > messages". Is
> > > > >> your
> > > > >> > > > thinking
> > > > >> > > > > > > > that
> > > > >> > > > > > > > > > we
> > > > >> > > > > > > > > > > > will
> > > > >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > > >> > > > UpdateMetadata only
> > > > >> > > > > > > > and
> > > > >> > > > > > > > > > add
> > > > >> > > > > > > > > > > > > topic configuration in this state?
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Hi Jose,
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Thanks for taking a look.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to
> > replace both
> > > > >> > > > > > > > > > LeaderAndIsrRequest
> > > > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic
> > configurations
> > > > >> would be
> > > > >> > > > fetched
> > > > >> > > > > > > > > along
> > > > >> > > > > > > > > > > > with the other metadata.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In the section "Broker Metadata
> > Management", you
> > > > >> mention
> > > > >> > > > "Just
> > > > >> > > > > > > > like
> > > > >> > > > > > > > > > > with
> > > > >> > > > > > > > > > > > a
> > > > >> > > > > > > > > > > > > fetch request, the broker will track the
> > offset
> > > > >> of the
> > > > >> > > > last
> > > > >> > > > > > > > updates
> > > > >> > > > > > > > > > it
> > > > >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> > > > >> requires that
> > > > >> > > > the
> > > > >> > > > > > > > > followers
> > > > >> > > > > > > > > > > > keep
> > > > >> > > > > > > > > > > > > all of the log entries (term/epoch and
> > offset)
> > > > >> that are
> > > > >> > > > after the
> > > > >> > > > > > > > > > > > > highwatermark. Any log entry before the
> > > > >> highwatermark
> > > > >> > > > can be
> > > > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > > > >> MetadataFetch API
> > > > >> > > > to only
> > > > >> > > > > > > > > return
> > > > >> > > > > > > > > > > log
> > > > >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike
> > the Raft
> > > > >> > > > replication API
> > > > >> > > > > > > > > > which
> > > > >> > > > > > > > > > > > > will replicate/fetch log entries after the
> > > > >> highwatermark
> > > > >> > > > for
> > > > >> > > > > > > > > > consensus?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> > > > >> metadata
> > > > >> > > > updates to
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > > > brokers until they've been stored on a
> > majority of
> > > > >> the
> > > > >> > > > Raft nodes.
> > > > >> > > > > > > > > The
> > > > >> > > > > > > > > > > > most obvious way to do that, like you
> > mentioned, is
> > > > >> to
> > > > >> > > > have the
> > > > >> > > > > > > > > brokers
> > > > >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.
> > There
> > > > >> might be
> > > > >> > > > a more
> > > > >> > > > > > > > > clever
> > > > >> > > > > > > > > > > way
> > > > >> > > > > > > > > > > > to do it by fetching the data, but not having
> > the
> > > > >> brokers
> > > > >> > > > act on it
> > > > >> > > > > > > > > > until
> > > > >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's
> > worth it
> > > > >> or
> > > > >> > > > not.  We'll
> > > > >> > > > > > > > > > discuss
> > > > >> > > > > > > > > > > > this more in a separate KIP that just
> > discusses
> > > > >> just Raft.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > > > >> mention "the
> > > > >> > > > > > > > > controller
> > > > >> > > > > > > > > > > will
> > > > >> > > > > > > > > > > > > send a full metadata image rather than a
> > series of
> > > > >> > > > deltas". This
> > > > >> > > > > > > > > KIP
> > > > >> > > > > > > > > > > > > doesn't go into the set of operations that
> > need
> > > > >> to be
> > > > >> > > > supported
> > > > >> > > > > > > > on
> > > > >> > > > > > > > > > top
> > > > >> > > > > > > > > > > of
> > > > >> > > > > > > > > > > > > Raft but it would be interested if this
> > "full
> > > > >> metadata
> > > > >> > > > image"
> > > > >> > > > > > > > could
> > > > >> > > > > > > > > > be
> > > > >> > > > > > > > > > > > > express also as deltas. For example,
> > assuming we
> > > > >> are
> > > > >> > > > replicating
> > > > >> > > > > > > > a
> > > > >> > > > > > > > > > map
> > > > >> > > > > > > > > > > > this
> > > > >> > > > > > > > > > > > > "full metadata image" could be a sequence
> > of "put"
> > > > >> > > > operations
> > > > >> > > > > > > > > (znode
> > > > >> > > > > > > > > > > > create
> > > > >> > > > > > > > > > > > > to borrow ZK semantics).
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > The full image can definitely be expressed as
> > a sum
> > > > >> of
> > > > >> > > > deltas.  At
> > > > >> > > > > > > > > some
> > > > >> > > > > > > > > > > > point, the number of deltas will get large
> > enough
> > > > >> that
> > > > >> > > > sending a
> > > > >> > > > > > > > full
> > > > >> > > > > > > > > > > image
> > > > >> > > > > > > > > > > > is better, though.  One question that we're
> > still
> > > > >> thinking
> > > > >> > > > about is
> > > > >> > > > > > > > > how
> > > > >> > > > > > > > > > > > much of this can be shared with generic Kafka
> > log
> > > > >> code,
> > > > >> > > > and how
> > > > >> > > > > > > > much
> > > > >> > > > > > > > > > > should
> > > > >> > > > > > > > > > > > be different.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > > > >> mention
> > > > >> > > > "This
> > > > >> > > > > > > > request
> > > > >> > > > > > > > > > will
> > > > >> > > > > > > > > > > > > double as a heartbeat, letting the
> > controller
> > > > >> know that
> > > > >> > > > the
> > > > >> > > > > > > > broker
> > > > >> > > > > > > > > is
> > > > >> > > > > > > > > > > > > alive". In section "Broker State Machine",
> > you
> > > > >> mention
> > > > >> > > > "The
> > > > >> > > > > > > > > > > MetadataFetch
> > > > >> > > > > > > > > > > > > API serves as this registration mechanism".
> > Does
> > > > >> this
> > > > >> > > > mean that
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > > > > MetadataFetch Request will optionally
> > include
> > > > >> broker
> > > > >> > > > > > > > configuration
> > > > >> > > > > > > > > > > > > information?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > I was originally thinking that the
> > > > >> MetadataFetchRequest
> > > > >> > > > should
> > > > >> > > > > > > > > include
> > > > >> > > > > > > > > > > > broker configuration information.  Thinking
> > about
> > > > >> this
> > > > >> > > > more, maybe
> > > > >> > > > > > > > we
> > > > >> > > > > > > > > > > > should just have a special registration RPC
> > that
> > > > >> contains
> > > > >> > > > that
> > > > >> > > > > > > > > > > information,
> > > > >> > > > > > > > > > > > to avoid sending it over the wire all the
> > time.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > Does this also mean that MetadataFetch
> > request
> > > > >> will
> > > > >> > > > result in
> > > > >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > > > >> replication
> > > > >> > > > protocol
> > > > >> > > > > > > > > before
> > > > >> > > > > > > > > > > you
> > > > >> > > > > > > > > > > > > can send the associated MetadataFetch
> > Response?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > I think we should require the broker to be
> > out of
> > > > >> the
> > > > >> > > > Offline state
> > > > >> > > > > > > > > > > before
> > > > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the
> > separate
> > > > >> > > > registration
> > > > >> > > > > > > > RPC
> > > > >> > > > > > > > > > > > should have completed first.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Broker State", you mention that
> > a
> > > > >> broker can
> > > > >> > > > > > > > transition
> > > > >> > > > > > > > > > to
> > > > >> > > > > > > > > > > > > online after it is caught with the
> > metadata. What
> > > > >> do you
> > > > >> > > > mean by
> > > > >> > > > > > > > > > this?
> > > > >> > > > > > > > > > > > > Metadata is always changing. How does the
> > broker
> > > > >> know
> > > > >> > > > that it is
> > > > >> > > > > > > > > > caught
> > > > >> > > > > > > > > > > > up
> > > > >> > > > > > > > > > > > > since it doesn't participate in the
> > consensus or
> > > > >> the
> > > > >> > > > advancement
> > > > >> > > > > > > > of
> > > > >> > > > > > > > > > the
> > > > >> > > > > > > > > > > > > highwatermark?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > That's a good point.  Being "caught up" is
> > somewhat
> > > > >> of a
> > > > >> > > > fuzzy
> > > > >> > > > > > > > > concept
> > > > >> > > > > > > > > > > > here, since the brokers do not participate in
> > the
> > > > >> metadata
> > > > >> > > > > > > > consensus.
> > > > >> > > > > > > > > > I
> > > > >> > > > > > > > > > > > think ideally we would want to define it in
> > terms
> > > > >> of time
> > > > >> > > > ("the
> > > > >> > > > > > > > > broker
> > > > >> > > > > > > > > > > has
> > > > >> > > > > > > > > > > > all the updates from the last 2 minutes", for
> > > > >> example.)
> > > > >> > > > We should
> > > > >> > > > > > > > > > spell
> > > > >> > > > > > > > > > > > this out better in the KIP.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Start the controller quorum
> > nodes",
> > > > >> you
> > > > >> > > > mention "Once
> > > > >> > > > > > > > > it
> > > > >> > > > > > > > > > > has
> > > > >> > > > > > > > > > > > > taken over the /controller node, the active
> > > > >> controller
> > > > >> > > > will
> > > > >> > > > > > > > proceed
> > > > >> > > > > > > > > > to
> > > > >> > > > > > > > > > > > load
> > > > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will write
> > out
> > > > >> this
> > > > >> > > > information
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > > the
> > > > >> > > > > > > > > > > > > quorum's metadata storage.  After this
> > point, the
> > > > >> > > > metadata quorum
> > > > >> > > > > > > > > > will
> > > > >> > > > > > > > > > > be
> > > > >> > > > > > > > > > > > > the metadata store of record, rather than
> > the
> > > > >> data in
> > > > >> > > > ZooKeeper."
> > > > >> > > > > > > > > > > During
> > > > >> > > > > > > > > > > > > this migration do should we expect to have a
> > > > >> small period
> > > > >> > > > > > > > > controller
> > > > >> > > > > > > > > > > > > unavailability while the controller
> > replicas this
> > > > >> state
> > > > >> > > > to all of
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > > > raft
> > > > >> > > > > > > > > > > > > nodes in the controller quorum and we
> > buffer new
> > > > >> > > > controller API
> > > > >> > > > > > > > > > > requests?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Yes, the controller would be unavailable
> > during this
> > > > >> > > > time.  I don't
> > > > >> > > > > > > > > > think
> > > > >> > > > > > > > > > > > this will be that different from the current
> > period
> > > > >> of
> > > > >> > > > > > > > unavailability
> > > > >> > > > > > > > > > > when
> > > > >> > > > > > > > > > > > a new controller starts up and needs to load
> > the
> > > > >> full
> > > > >> > > > state from
> > > > >> > > > > > > > ZK.
> > > > >> > > > > > > > > > The
> > > > >> > > > > > > > > > > > main difference is that in this period, we'd
> > have
> > > > >> to write
> > > > >> > > > to the
> > > > >> > > > > > > > > > > > controller quorum rather than just to
> > memory.  But
> > > > >> we
> > > > >> > > > believe this
> > > > >> > > > > > > > > > should
> > > > >> > > > > > > > > > > > be pretty fast.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > regards,
> > > > >> > > > > > > > > > > > Colin
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > Thanks!
> > > > >> > > > > > > > > > > > > -Jose
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> > >
> > > > >> > > --
> > > > >> > > David Arthur
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ryanne Dolan <ry...@gmail.com>.
Colin, have you considered leveraging Apache Ratis (incubating)?

Ryanne

On Wed, Aug 21, 2019, 1:28 PM Colin McCabe <cm...@apache.org> wrote:

> On Wed, Aug 21, 2019, at 06:38, Eno Thereska wrote:
> > Hi Colin,
> >
> > Nice KIP! For such a big change it would be good to add a pointer or
> > two to related work that provides some sort of soft proof that the
> > approach taken makes sense. Also such work often builds on other work
> > and it might be useful to trace its roots. May I recommend adding a
> > pointer to "Tango: Distributed Data Structures over a Shared Log"
> > (http://www.cs.cornell.edu/~taozou/sosp13/tangososp.pdf)? There are
> > other papers that are related (e.g., a more recent one one "The
> > FuzzyLog: A Partially Ordered Shared Log"
> > (https://www.usenix.org/system/files/osdi18-lockerman.pdf)).
> >
> > Both papers would add to the strength of your motivation.
> >
>
> Hi Eno,
>
> Good point.  I added a "references" section on the end and added the Tango
> paper.  I am not sure we need the FuzzyLog one, though.
>
> I also added a link to the Raft paper and one of the papers on HDFS, since
> I feel like these are very relevant here.
>
> best,
> Colin
>
> > Cheers
> > Eno
> >
> > On Wed, Aug 21, 2019 at 12:22 PM Ron Dagostino <rn...@gmail.com>
> wrote:
> > >
> > > Hi Colin.  I like the concept of a "bridge release" for migrating off
> of
> > > Zookeeper, but I worry that it may become a bottleneck if people
> hesitate
> > > to replace Zookeeper -- they would be unable to adopt newer versions of
> > > Kafka until taking (what feels to them like) a giant leap.  As an
> example,
> > > assuming version 4.0.x of Kafka is the supported bridge release, I
> would
> > > not be surprised if uptake of the 4.x release and the time-based
> releases
> > > that follow it end up being much slower due to the perceived barrier.
> > >
> > > Any perceived barrier could be lowered if the 4.0.x release could
> > > optionally continue to use Zookeeper -- then the cutover would be two
> > > incremental steps (move to 4.0.x, then replace Zookeeper while staying
> on
> > > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> > > Zookeeper in one fell swoop).
> > >
> > > Regardless of whether what I wrote above has merit or not, I think the
> KIP
> > > should be more explicit about what the upgrade constraints actually
> are.
> > > Can the bridge release be adopted with Zookeeper remaining in place and
> > > then cutting over as a second, follow-on step, or must the Controller
> > > Quorum nodes be started first and the bridge release cannot be used
> with
> > > Zookeeper at all?  If the bridge release cannot be used with Zookeeper
> at
> > > all, then no version at or beyond the bridge release is available
> > > unless/until abandoning Zookeeper; if the bridge release can be used
> with
> > > Zookeeper, then is it the only version that can be used with
> Zookeeper, or
> > > can Zookeeper be kept for additional releases if desired?
> > >
> > > Ron
> > >
> > > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com>
> wrote:
> > >
> > > > Hi Colin.  The diagram up at the top confused me -- specifically, the
> > > > lines connecting the controller/active-controller to the brokers.  I
> had
> > > > assumed the arrows on those lines represented the direction of data
> flow,
> > > > but that is not the case; the arrows actually identify the target of
> the
> > > > action, and the non-arrowed end indicates the initiator of the
> action.  For
> > > > example, the lines point from the controller to the brokers in the
> "today"
> > > > section on the left to show that the controller pushes to the
> brokers; the
> > > > lines point from the brokers to the active-controller in the
> "tomorrow"
> > > > section on the right to show that the brokers pull from the
> > > > active-controller.  As I said, this confused me because my gut
> instinct was
> > > > to interpret the arrow as indicating the direction of data flow, and
> when I
> > > > look at the "tomorrow" picture on the right I initially thought
> information
> > > > was moving from the brokers to the active-controller.  Did you
> consider
> > > > drawing that picture with the arrows reversed in the "tomorrow" side
> so
> > > > that the arrows represent the direction of data flow, and then add
> the
> > > > labels "push" on the "today" side and "pull" on the "tomorrow" side
> to
> > > > indicate who initiates the data flow?  It occurs to me that this
> picture
> > > > may end up being widely distributed, so it might be in everyone's
> interest
> > > > to proactively avoid any possible confusion by being more explicit.
> > > >
> > > > Minor corrections?
> > > > <<<In the current world, a broker which can contact ZooKeeper but
> which
> > > > is partitioned from the active controller
> > > > >>>In the current world, a broker which can contact ZooKeeper but
> which
> > > > is partitioned from the controller
> > > >
> > > > <<<Eventually, the controller will ask the broker to finally go
> offline
> > > > >>>Eventually, the active controller will ask the broker to finally
> go
> > > > offline
> > > >
> > > > <<<New versions of the clients should send these operations directly
> to
> > > > the controller
> > > > >>>New versions of the clients should send these operations directly
> to
> > > > the active controller
> > > >
> > > > <<<In the post-ZK world, the leader will make an RPC to the
> controller
> > > > instead
> > > > >>>In the post-ZK world, the leader will make an RPC to the active
> > > > controller instead
> > > >
> > > > <<<For example, the brokers may need to forward their requests to the
> > > > controller.
> > > > >>>For example, the brokers may need to forward their requests to the
> > > > active controller.
> > > >
> > > > <<<The new controller will monitor ZooKeeper for legacy broker node
> > > > registrations
> > > > >>>The new (active) controller will monitor ZooKeeper for legacy
> broker
> > > > node registrations
> > > >
> > > > Ron
> > > >
> > > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org>
> wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> The KIP has been out for a while, so I'm thinking about calling a
> vote
> > > >> some time this week.
> > > >>
> > > >> best,
> > > >> Colin
> > > >>
> > > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > >> > > Thanks for the KIP, Colin. This looks great!
> > > >> > >
> > > >> > > I really like the idea of separating the Controller and Broker
> JVMs.
> > > >> > >
> > > >> > > As you alluded to above, it might be nice to have a separate
> > > >> > > broker-registration API to avoid overloading the metadata fetch
> API.
> > > >> > >
> > > >> >
> > > >> > Hi David,
> > > >> >
> > > >> > Thanks for taking a look.
> > > >> >
> > > >> > I removed the sentence about MetadataFetch also serving as the
> broker
> > > >> > registration API.  I think I agree that we will probably want a
> > > >> > separate RPC to fill this role.  We will have a follow-on KIP
> that will
> > > >> > go into more detail about metadata propagation and registration
> in the
> > > >> > post-ZK world.  That KIP will also have a full description of the
> > > >> > registration RPC, etc.  For now, I think the important part for
> KIP-500
> > > >> > is that the broker registers with the controller quorum.  On
> > > >> > registration, the controller quorum assigns it a new broker epoch,
> > > >> > which can distinguish successive broker incarnations.
> > > >> >
> > > >> > >
> > > >> > > When a broker gets a metadata delta, will it be a sequence of
> deltas
> > > >> since
> > > >> > > the last update or a cumulative delta since the last update?
> > > >> > >
> > > >> >
> > > >> > It will be a sequence of deltas.  Basically, the broker will be
> reading
> > > >> > from the metadata log.
> > > >> >
> > > >> > >
> > > >> > > Will we include any kind of integrity check on the deltas to
> ensure
> > > >> the brokers
> > > >> > > have applied them correctly? Perhaps this will be addressed in
> one of
> > > >> the
> > > >> > > follow-on KIPs.
> > > >> > >
> > > >> >
> > > >> > In general, we will have checksums on the metadata that we
> fetch.  This
> > > >> > is similar to how we have checksums on regular data.  Or if the
> > > >> > question is about catching logic errors in the metadata handling
> code,
> > > >> > that sounds more like something that should be caught by test
> cases.
> > > >> >
> > > >> > best,
> > > >> > Colin
> > > >> >
> > > >> >
> > > >> > > Thanks!
> > > >> > >
> > > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cmccabe@apache.org
> >
> > > >> wrote:
> > > >> > >
> > > >> > > > Hi Mickael,
> > > >> > > >
> > > >> > > > Thanks for taking a look.
> > > >> > > >
> > > >> > > > I don't think we want to support that kind of multi-tenancy
> at the
> > > >> > > > controller level.  If the cluster is small enough that we
> want to
> > > >> pack the
> > > >> > > > controller(s) with something else, we could run them
> alongside the
> > > >> brokers,
> > > >> > > > or possibly inside three of the broker JVMs.
> > > >> > > >
> > > >> > > > best,
> > > >> > > > Colin
> > > >> > > >
> > > >> > > >
> > > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > >> > > > > Thank Colin for kickstarting this initiative.
> > > >> > > > >
> > > >> > > > > Just one question.
> > > >> > > > > - A nice feature of Zookeeper is the ability to use chroots
> and
> > > >> have
> > > >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is
> this
> > > >> > > > > something we should keep?
> > > >> > > > >
> > > >> > > > > Thanks
> > > >> > > > >
> > > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> cmccabe@apache.org>
> > > >> wrote:
> > > >> > > > > >
> > > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > >> > > > > > > Hi Colin,
> > > >> > > > > > >
> > > >> > > > > > > Thanks for the KIP.
> > > >> > > > > > >
> > > >> > > > > > > Currently ZooKeeper provides a convenient notification
> > > >> mechanism for
> > > >> > > > > > > knowing that broker and topic configuration has
> changed. While
> > > >> > > > KIP-500 does
> > > >> > > > > > > suggest that incremental metadata update is expected to
> come
> > > >> to
> > > >> > > > clients
> > > >> > > > > > > eventually, that would seem to imply that for some
> number of
> > > >> > > > releases there
> > > >> > > > > > > would be no equivalent mechanism for knowing about
> config
> > > >> changes.
> > > >> > > > Is there
> > > >> > > > > > > any thinking at this point about how a similar
> notification
> > > >> might be
> > > >> > > > > > > provided in the future?
> > > >> > > > > >
> > > >> > > > > > We could eventually have some inotify-like mechanism where
> > > >> clients
> > > >> > > > could register interest in various types of events and got
> notified
> > > >> when
> > > >> > > > they happened.  Reading the metadata log is conceptually
> simple.
> > > >> The main
> > > >> > > > complexity would be in setting up an API that made sense and
> that
> > > >> didn't
> > > >> > > > unduly constrain future implementations.  We'd have to think
> > > >> carefully
> > > >> > > > about what the real use-cases for this were, though.
> > > >> > > > > >
> > > >> > > > > > best,
> > > >> > > > > > Colin
> > > >> > > > > >
> > > >> > > > > > >
> > > >> > > > > > > Thanks,
> > > >> > > > > > >
> > > >> > > > > > > Tom
> > > >> > > > > > >
> > > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > >> > > > viktorsomogyi@gmail.com>
> > > >> > > > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hey Colin,
> > > >> > > > > > > >
> > > >> > > > > > > > I think this is a long-awaited KIP, thanks for
> driving it.
> > > >> I'm
> > > >> > > > excited to
> > > >> > > > > > > > see this in Kafka once. I collected my questions (and
> I
> > > >> accept the
> > > >> > > > "TBD"
> > > >> > > > > > > > answer as they might be a bit deep for this high
> level :) ).
> > > >> > > > > > > > 1.) Are there any specific reasons for the Controller
> just
> > > >> > > > periodically
> > > >> > > > > > > > persisting its state on disk periodically instead of
> > > >> > > > asynchronously with
> > > >> > > > > > > > every update? Wouldn't less frequent saves increase
> the
> > > >> chance for
> > > >> > > > missing
> > > >> > > > > > > > a state change if the controller crashes between two
> saves?
> > > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from
> the
> > > >> follower
> > > >> > > > > > > > controllers? I assume that followers would have
> up-to-date
> > > >> > > > information
> > > >> > > > > > > > therefore brokers could fetch from there in theory.
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > > Viktor
> > > >> > > > > > > >
> > > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > >> > > > reluctanthero104@gmail.com>
> > > >> > > > > > > > wrote:
> > > >> > > > > > > >
> > > >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> > > >> follow-up KIPs
> > > >> > > > sounds
> > > >> > > > > > > > like
> > > >> > > > > > > > > a good idea.
> > > >> > > > > > > > >
> > > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > > >> ismael@juma.me.uk>
> > > >> > > > wrote:
> > > >> > > > > > > > >
> > > >> > > > > > > > > > Hi Boyang,
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > Yes, there will be several KIPs that will discuss
> the
> > > >> items you
> > > >> > > > > > > > describe
> > > >> > > > > > > > > in
> > > >> > > > > > > > > > detail. Colin, it may be helpful to make this
> clear in
> > > >> the KIP
> > > >> > > > 500
> > > >> > > > > > > > > > description.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > Ismael
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > >> > > > reluctanthero104@gmail.com
> > > >> > > > > > > > >
> > > >> > > > > > > > > > wrote:
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > > Thanks Colin for initiating this important
> effort!
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > One question I have is whether we have a session
> > > >> discussing
> > > >> > > > the
> > > >> > > > > > > > > > controller
> > > >> > > > > > > > > > > failover in the new architecture? I know we are
> using
> > > >> Raft
> > > >> > > > protocol
> > > >> > > > > > > > to
> > > >> > > > > > > > > > > failover, yet it's still valuable to discuss the
> > > >> steps new
> > > >> > > > cluster is
> > > >> > > > > > > > > > going
> > > >> > > > > > > > > > > to take to reach the stable stage again, so
> that we
> > > >> could
> > > >> > > > easily
> > > >> > > > > > > > > measure
> > > >> > > > > > > > > > > the availability of the metadata servers.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > Another suggestion I have is to write a
> step-by-step
> > > >> design
> > > >> > > > doc like
> > > >> > > > > > > > > what
> > > >> > > > > > > > > > > we did in KIP-98
> > > >> > > > > > > > > > > <
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > >
> > > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >> > > > > > > > > > > >,
> > > >> > > > > > > > > > > including the new request protocols and how
> they are
> > > >> > > > interacting in
> > > >> > > > > > > > the
> > > >> > > > > > > > > > new
> > > >> > > > > > > > > > > cluster. For a complicated change like this, an
> > > >> > > > implementation design
> > > >> > > > > > > > > doc
> > > >> > > > > > > > > > > help a lot in the review process, otherwise most
> > > >> discussions
> > > >> > > > we have
> > > >> > > > > > > > > will
> > > >> > > > > > > > > > > focus on high level and lose important details
> as we
> > > >> > > > discover them in
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > > post-agreement phase.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > Boyang
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > >> > > > cmccabe@apache.org>
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando
> Garcia
> > > >> Sancio
> > > >> > > > wrote:
> > > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a
> few
> > > >> comments
> > > >> > > > and
> > > >> > > > > > > > > questions.
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> > > >> mentioned the
> > > >> > > > > > > > LeaderAndIsr
> > > >> > > > > > > > > > and
> > > >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates
> which
> > > >> the
> > > >> > > > controller
> > > >> > > > > > > > > > pushes,
> > > >> > > > > > > > > > > > such
> > > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata
> messages". Is
> > > >> your
> > > >> > > > thinking
> > > >> > > > > > > > that
> > > >> > > > > > > > > > we
> > > >> > > > > > > > > > > > will
> > > >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > >> > > > UpdateMetadata only
> > > >> > > > > > > > and
> > > >> > > > > > > > > > add
> > > >> > > > > > > > > > > > > topic configuration in this state?
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Hi Jose,
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Thanks for taking a look.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to
> replace both
> > > >> > > > > > > > > > LeaderAndIsrRequest
> > > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic
> configurations
> > > >> would be
> > > >> > > > fetched
> > > >> > > > > > > > > along
> > > >> > > > > > > > > > > > with the other metadata.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In the section "Broker Metadata
> Management", you
> > > >> mention
> > > >> > > > "Just
> > > >> > > > > > > > like
> > > >> > > > > > > > > > > with
> > > >> > > > > > > > > > > > a
> > > >> > > > > > > > > > > > > fetch request, the broker will track the
> offset
> > > >> of the
> > > >> > > > last
> > > >> > > > > > > > updates
> > > >> > > > > > > > > > it
> > > >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> > > >> requires that
> > > >> > > > the
> > > >> > > > > > > > > followers
> > > >> > > > > > > > > > > > keep
> > > >> > > > > > > > > > > > > all of the log entries (term/epoch and
> offset)
> > > >> that are
> > > >> > > > after the
> > > >> > > > > > > > > > > > > highwatermark. Any log entry before the
> > > >> highwatermark
> > > >> > > > can be
> > > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > > >> MetadataFetch API
> > > >> > > > to only
> > > >> > > > > > > > > return
> > > >> > > > > > > > > > > log
> > > >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike
> the Raft
> > > >> > > > replication API
> > > >> > > > > > > > > > which
> > > >> > > > > > > > > > > > > will replicate/fetch log entries after the
> > > >> highwatermark
> > > >> > > > for
> > > >> > > > > > > > > > consensus?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> > > >> metadata
> > > >> > > > updates to
> > > >> > > > > > > > the
> > > >> > > > > > > > > > > > brokers until they've been stored on a
> majority of
> > > >> the
> > > >> > > > Raft nodes.
> > > >> > > > > > > > > The
> > > >> > > > > > > > > > > > most obvious way to do that, like you
> mentioned, is
> > > >> to
> > > >> > > > have the
> > > >> > > > > > > > > brokers
> > > >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.
> There
> > > >> might be
> > > >> > > > a more
> > > >> > > > > > > > > clever
> > > >> > > > > > > > > > > way
> > > >> > > > > > > > > > > > to do it by fetching the data, but not having
> the
> > > >> brokers
> > > >> > > > act on it
> > > >> > > > > > > > > > until
> > > >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's
> worth it
> > > >> or
> > > >> > > > not.  We'll
> > > >> > > > > > > > > > discuss
> > > >> > > > > > > > > > > > this more in a separate KIP that just
> discusses
> > > >> just Raft.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > > >> mention "the
> > > >> > > > > > > > > controller
> > > >> > > > > > > > > > > will
> > > >> > > > > > > > > > > > > send a full metadata image rather than a
> series of
> > > >> > > > deltas". This
> > > >> > > > > > > > > KIP
> > > >> > > > > > > > > > > > > doesn't go into the set of operations that
> need
> > > >> to be
> > > >> > > > supported
> > > >> > > > > > > > on
> > > >> > > > > > > > > > top
> > > >> > > > > > > > > > > of
> > > >> > > > > > > > > > > > > Raft but it would be interested if this
> "full
> > > >> metadata
> > > >> > > > image"
> > > >> > > > > > > > could
> > > >> > > > > > > > > > be
> > > >> > > > > > > > > > > > > express also as deltas. For example,
> assuming we
> > > >> are
> > > >> > > > replicating
> > > >> > > > > > > > a
> > > >> > > > > > > > > > map
> > > >> > > > > > > > > > > > this
> > > >> > > > > > > > > > > > > "full metadata image" could be a sequence
> of "put"
> > > >> > > > operations
> > > >> > > > > > > > > (znode
> > > >> > > > > > > > > > > > create
> > > >> > > > > > > > > > > > > to borrow ZK semantics).
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > The full image can definitely be expressed as
> a sum
> > > >> of
> > > >> > > > deltas.  At
> > > >> > > > > > > > > some
> > > >> > > > > > > > > > > > point, the number of deltas will get large
> enough
> > > >> that
> > > >> > > > sending a
> > > >> > > > > > > > full
> > > >> > > > > > > > > > > image
> > > >> > > > > > > > > > > > is better, though.  One question that we're
> still
> > > >> thinking
> > > >> > > > about is
> > > >> > > > > > > > > how
> > > >> > > > > > > > > > > > much of this can be shared with generic Kafka
> log
> > > >> code,
> > > >> > > > and how
> > > >> > > > > > > > much
> > > >> > > > > > > > > > > should
> > > >> > > > > > > > > > > > be different.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > > >> mention
> > > >> > > > "This
> > > >> > > > > > > > request
> > > >> > > > > > > > > > will
> > > >> > > > > > > > > > > > > double as a heartbeat, letting the
> controller
> > > >> know that
> > > >> > > > the
> > > >> > > > > > > > broker
> > > >> > > > > > > > > is
> > > >> > > > > > > > > > > > > alive". In section "Broker State Machine",
> you
> > > >> mention
> > > >> > > > "The
> > > >> > > > > > > > > > > MetadataFetch
> > > >> > > > > > > > > > > > > API serves as this registration mechanism".
> Does
> > > >> this
> > > >> > > > mean that
> > > >> > > > > > > > the
> > > >> > > > > > > > > > > > > MetadataFetch Request will optionally
> include
> > > >> broker
> > > >> > > > > > > > configuration
> > > >> > > > > > > > > > > > > information?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > I was originally thinking that the
> > > >> MetadataFetchRequest
> > > >> > > > should
> > > >> > > > > > > > > include
> > > >> > > > > > > > > > > > broker configuration information.  Thinking
> about
> > > >> this
> > > >> > > > more, maybe
> > > >> > > > > > > > we
> > > >> > > > > > > > > > > > should just have a special registration RPC
> that
> > > >> contains
> > > >> > > > that
> > > >> > > > > > > > > > > information,
> > > >> > > > > > > > > > > > to avoid sending it over the wire all the
> time.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > > Does this also mean that MetadataFetch
> request
> > > >> will
> > > >> > > > result in
> > > >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > > >> replication
> > > >> > > > protocol
> > > >> > > > > > > > > before
> > > >> > > > > > > > > > > you
> > > >> > > > > > > > > > > > > can send the associated MetadataFetch
> Response?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > I think we should require the broker to be
> out of
> > > >> the
> > > >> > > > Offline state
> > > >> > > > > > > > > > > before
> > > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the
> separate
> > > >> > > > registration
> > > >> > > > > > > > RPC
> > > >> > > > > > > > > > > > should have completed first.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Broker State", you mention that
> a
> > > >> broker can
> > > >> > > > > > > > transition
> > > >> > > > > > > > > > to
> > > >> > > > > > > > > > > > > online after it is caught with the
> metadata. What
> > > >> do you
> > > >> > > > mean by
> > > >> > > > > > > > > > this?
> > > >> > > > > > > > > > > > > Metadata is always changing. How does the
> broker
> > > >> know
> > > >> > > > that it is
> > > >> > > > > > > > > > caught
> > > >> > > > > > > > > > > > up
> > > >> > > > > > > > > > > > > since it doesn't participate in the
> consensus or
> > > >> the
> > > >> > > > advancement
> > > >> > > > > > > > of
> > > >> > > > > > > > > > the
> > > >> > > > > > > > > > > > > highwatermark?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > That's a good point.  Being "caught up" is
> somewhat
> > > >> of a
> > > >> > > > fuzzy
> > > >> > > > > > > > > concept
> > > >> > > > > > > > > > > > here, since the brokers do not participate in
> the
> > > >> metadata
> > > >> > > > > > > > consensus.
> > > >> > > > > > > > > > I
> > > >> > > > > > > > > > > > think ideally we would want to define it in
> terms
> > > >> of time
> > > >> > > > ("the
> > > >> > > > > > > > > broker
> > > >> > > > > > > > > > > has
> > > >> > > > > > > > > > > > all the updates from the last 2 minutes", for
> > > >> example.)
> > > >> > > > We should
> > > >> > > > > > > > > > spell
> > > >> > > > > > > > > > > > this out better in the KIP.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Start the controller quorum
> nodes",
> > > >> you
> > > >> > > > mention "Once
> > > >> > > > > > > > > it
> > > >> > > > > > > > > > > has
> > > >> > > > > > > > > > > > > taken over the /controller node, the active
> > > >> controller
> > > >> > > > will
> > > >> > > > > > > > proceed
> > > >> > > > > > > > > > to
> > > >> > > > > > > > > > > > load
> > > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will write
> out
> > > >> this
> > > >> > > > information
> > > >> > > > > > > > to
> > > >> > > > > > > > > > the
> > > >> > > > > > > > > > > > > quorum's metadata storage.  After this
> point, the
> > > >> > > > metadata quorum
> > > >> > > > > > > > > > will
> > > >> > > > > > > > > > > be
> > > >> > > > > > > > > > > > > the metadata store of record, rather than
> the
> > > >> data in
> > > >> > > > ZooKeeper."
> > > >> > > > > > > > > > > During
> > > >> > > > > > > > > > > > > this migration do should we expect to have a
> > > >> small period
> > > >> > > > > > > > > controller
> > > >> > > > > > > > > > > > > unavailability while the controller
> replicas this
> > > >> state
> > > >> > > > to all of
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > > > raft
> > > >> > > > > > > > > > > > > nodes in the controller quorum and we
> buffer new
> > > >> > > > controller API
> > > >> > > > > > > > > > > requests?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Yes, the controller would be unavailable
> during this
> > > >> > > > time.  I don't
> > > >> > > > > > > > > > think
> > > >> > > > > > > > > > > > this will be that different from the current
> period
> > > >> of
> > > >> > > > > > > > unavailability
> > > >> > > > > > > > > > > when
> > > >> > > > > > > > > > > > a new controller starts up and needs to load
> the
> > > >> full
> > > >> > > > state from
> > > >> > > > > > > > ZK.
> > > >> > > > > > > > > > The
> > > >> > > > > > > > > > > > main difference is that in this period, we'd
> have
> > > >> to write
> > > >> > > > to the
> > > >> > > > > > > > > > > > controller quorum rather than just to
> memory.  But
> > > >> we
> > > >> > > > believe this
> > > >> > > > > > > > > > should
> > > >> > > > > > > > > > > > be pretty fast.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > regards,
> > > >> > > > > > > > > > > > Colin
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > Thanks!
> > > >> > > > > > > > > > > > > -Jose
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> > >
> > > >> > > --
> > > >> > > David Arthur
> > > >> > >
> > > >> >
> > > >>
> > > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Wed, Aug 21, 2019, at 06:38, Eno Thereska wrote:
> Hi Colin,
> 
> Nice KIP! For such a big change it would be good to add a pointer or
> two to related work that provides some sort of soft proof that the
> approach taken makes sense. Also such work often builds on other work
> and it might be useful to trace its roots. May I recommend adding a
> pointer to "Tango: Distributed Data Structures over a Shared Log"
> (http://www.cs.cornell.edu/~taozou/sosp13/tangososp.pdf)? There are
> other papers that are related (e.g., a more recent one one "The
> FuzzyLog: A Partially Ordered Shared Log"
> (https://www.usenix.org/system/files/osdi18-lockerman.pdf)).
> 
> Both papers would add to the strength of your motivation.
> 

Hi Eno,

Good point.  I added a "references" section on the end and added the Tango paper.  I am not sure we need the FuzzyLog one, though.

I also added a link to the Raft paper and one of the papers on HDFS, since I feel like these are very relevant here.

best,
Colin

> Cheers
> Eno
> 
> On Wed, Aug 21, 2019 at 12:22 PM Ron Dagostino <rn...@gmail.com> wrote:
> >
> > Hi Colin.  I like the concept of a "bridge release" for migrating off of
> > Zookeeper, but I worry that it may become a bottleneck if people hesitate
> > to replace Zookeeper -- they would be unable to adopt newer versions of
> > Kafka until taking (what feels to them like) a giant leap.  As an example,
> > assuming version 4.0.x of Kafka is the supported bridge release, I  would
> > not be surprised if uptake of the 4.x release and the time-based releases
> > that follow it end up being much slower due to the perceived barrier.
> >
> > Any perceived barrier could be lowered if the 4.0.x release could
> > optionally continue to use Zookeeper -- then the cutover would be two
> > incremental steps (move to 4.0.x, then replace Zookeeper while staying on
> > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> > Zookeeper in one fell swoop).
> >
> > Regardless of whether what I wrote above has merit or not, I think the KIP
> > should be more explicit about what the upgrade constraints actually are.
> > Can the bridge release be adopted with Zookeeper remaining in place and
> > then cutting over as a second, follow-on step, or must the Controller
> > Quorum nodes be started first and the bridge release cannot be used with
> > Zookeeper at all?  If the bridge release cannot be used with Zookeeper at
> > all, then no version at or beyond the bridge release is available
> > unless/until abandoning Zookeeper; if the bridge release can be used with
> > Zookeeper, then is it the only version that can be used with Zookeeper, or
> > can Zookeeper be kept for additional releases if desired?
> >
> > Ron
> >
> > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com> wrote:
> >
> > > Hi Colin.  The diagram up at the top confused me -- specifically, the
> > > lines connecting the controller/active-controller to the brokers.  I had
> > > assumed the arrows on those lines represented the direction of data flow,
> > > but that is not the case; the arrows actually identify the target of the
> > > action, and the non-arrowed end indicates the initiator of the action.  For
> > > example, the lines point from the controller to the brokers in the "today"
> > > section on the left to show that the controller pushes to the brokers; the
> > > lines point from the brokers to the active-controller in the "tomorrow"
> > > section on the right to show that the brokers pull from the
> > > active-controller.  As I said, this confused me because my gut instinct was
> > > to interpret the arrow as indicating the direction of data flow, and when I
> > > look at the "tomorrow" picture on the right I initially thought information
> > > was moving from the brokers to the active-controller.  Did you consider
> > > drawing that picture with the arrows reversed in the "tomorrow" side so
> > > that the arrows represent the direction of data flow, and then add the
> > > labels "push" on the "today" side and "pull" on the "tomorrow" side to
> > > indicate who initiates the data flow?  It occurs to me that this picture
> > > may end up being widely distributed, so it might be in everyone's interest
> > > to proactively avoid any possible confusion by being more explicit.
> > >
> > > Minor corrections?
> > > <<<In the current world, a broker which can contact ZooKeeper but which
> > > is partitioned from the active controller
> > > >>>In the current world, a broker which can contact ZooKeeper but which
> > > is partitioned from the controller
> > >
> > > <<<Eventually, the controller will ask the broker to finally go offline
> > > >>>Eventually, the active controller will ask the broker to finally go
> > > offline
> > >
> > > <<<New versions of the clients should send these operations directly to
> > > the controller
> > > >>>New versions of the clients should send these operations directly to
> > > the active controller
> > >
> > > <<<In the post-ZK world, the leader will make an RPC to the controller
> > > instead
> > > >>>In the post-ZK world, the leader will make an RPC to the active
> > > controller instead
> > >
> > > <<<For example, the brokers may need to forward their requests to the
> > > controller.
> > > >>>For example, the brokers may need to forward their requests to the
> > > active controller.
> > >
> > > <<<The new controller will monitor ZooKeeper for legacy broker node
> > > registrations
> > > >>>The new (active) controller will monitor ZooKeeper for legacy broker
> > > node registrations
> > >
> > > Ron
> > >
> > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > >> Hi all,
> > >>
> > >> The KIP has been out for a while, so I'm thinking about calling a vote
> > >> some time this week.
> > >>
> > >> best,
> > >> Colin
> > >>
> > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > >> > > Thanks for the KIP, Colin. This looks great!
> > >> > >
> > >> > > I really like the idea of separating the Controller and Broker JVMs.
> > >> > >
> > >> > > As you alluded to above, it might be nice to have a separate
> > >> > > broker-registration API to avoid overloading the metadata fetch API.
> > >> > >
> > >> >
> > >> > Hi David,
> > >> >
> > >> > Thanks for taking a look.
> > >> >
> > >> > I removed the sentence about MetadataFetch also serving as the broker
> > >> > registration API.  I think I agree that we will probably want a
> > >> > separate RPC to fill this role.  We will have a follow-on KIP that will
> > >> > go into more detail about metadata propagation and registration in the
> > >> > post-ZK world.  That KIP will also have a full description of the
> > >> > registration RPC, etc.  For now, I think the important part for KIP-500
> > >> > is that the broker registers with the controller quorum.  On
> > >> > registration, the controller quorum assigns it a new broker epoch,
> > >> > which can distinguish successive broker incarnations.
> > >> >
> > >> > >
> > >> > > When a broker gets a metadata delta, will it be a sequence of deltas
> > >> since
> > >> > > the last update or a cumulative delta since the last update?
> > >> > >
> > >> >
> > >> > It will be a sequence of deltas.  Basically, the broker will be reading
> > >> > from the metadata log.
> > >> >
> > >> > >
> > >> > > Will we include any kind of integrity check on the deltas to ensure
> > >> the brokers
> > >> > > have applied them correctly? Perhaps this will be addressed in one of
> > >> the
> > >> > > follow-on KIPs.
> > >> > >
> > >> >
> > >> > In general, we will have checksums on the metadata that we fetch.  This
> > >> > is similar to how we have checksums on regular data.  Or if the
> > >> > question is about catching logic errors in the metadata handling code,
> > >> > that sounds more like something that should be caught by test cases.
> > >> >
> > >> > best,
> > >> > Colin
> > >> >
> > >> >
> > >> > > Thanks!
> > >> > >
> > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
> > >> wrote:
> > >> > >
> > >> > > > Hi Mickael,
> > >> > > >
> > >> > > > Thanks for taking a look.
> > >> > > >
> > >> > > > I don't think we want to support that kind of multi-tenancy at the
> > >> > > > controller level.  If the cluster is small enough that we want to
> > >> pack the
> > >> > > > controller(s) with something else, we could run them alongside the
> > >> brokers,
> > >> > > > or possibly inside three of the broker JVMs.
> > >> > > >
> > >> > > > best,
> > >> > > > Colin
> > >> > > >
> > >> > > >
> > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > >> > > > > Thank Colin for kickstarting this initiative.
> > >> > > > >
> > >> > > > > Just one question.
> > >> > > > > - A nice feature of Zookeeper is the ability to use chroots and
> > >> have
> > >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is this
> > >> > > > > something we should keep?
> > >> > > > >
> > >> > > > > Thanks
> > >> > > > >
> > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org>
> > >> wrote:
> > >> > > > > >
> > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > >> > > > > > > Hi Colin,
> > >> > > > > > >
> > >> > > > > > > Thanks for the KIP.
> > >> > > > > > >
> > >> > > > > > > Currently ZooKeeper provides a convenient notification
> > >> mechanism for
> > >> > > > > > > knowing that broker and topic configuration has changed. While
> > >> > > > KIP-500 does
> > >> > > > > > > suggest that incremental metadata update is expected to come
> > >> to
> > >> > > > clients
> > >> > > > > > > eventually, that would seem to imply that for some number of
> > >> > > > releases there
> > >> > > > > > > would be no equivalent mechanism for knowing about config
> > >> changes.
> > >> > > > Is there
> > >> > > > > > > any thinking at this point about how a similar notification
> > >> might be
> > >> > > > > > > provided in the future?
> > >> > > > > >
> > >> > > > > > We could eventually have some inotify-like mechanism where
> > >> clients
> > >> > > > could register interest in various types of events and got notified
> > >> when
> > >> > > > they happened.  Reading the metadata log is conceptually simple.
> > >> The main
> > >> > > > complexity would be in setting up an API that made sense and that
> > >> didn't
> > >> > > > unduly constrain future implementations.  We'd have to think
> > >> carefully
> > >> > > > about what the real use-cases for this were, though.
> > >> > > > > >
> > >> > > > > > best,
> > >> > > > > > Colin
> > >> > > > > >
> > >> > > > > > >
> > >> > > > > > > Thanks,
> > >> > > > > > >
> > >> > > > > > > Tom
> > >> > > > > > >
> > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > >> > > > viktorsomogyi@gmail.com>
> > >> > > > > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hey Colin,
> > >> > > > > > > >
> > >> > > > > > > > I think this is a long-awaited KIP, thanks for driving it.
> > >> I'm
> > >> > > > excited to
> > >> > > > > > > > see this in Kafka once. I collected my questions (and I
> > >> accept the
> > >> > > > "TBD"
> > >> > > > > > > > answer as they might be a bit deep for this high level :) ).
> > >> > > > > > > > 1.) Are there any specific reasons for the Controller just
> > >> > > > periodically
> > >> > > > > > > > persisting its state on disk periodically instead of
> > >> > > > asynchronously with
> > >> > > > > > > > every update? Wouldn't less frequent saves increase the
> > >> chance for
> > >> > > > missing
> > >> > > > > > > > a state change if the controller crashes between two saves?
> > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from the
> > >> follower
> > >> > > > > > > > controllers? I assume that followers would have up-to-date
> > >> > > > information
> > >> > > > > > > > therefore brokers could fetch from there in theory.
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > > Viktor
> > >> > > > > > > >
> > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > >> > > > reluctanthero104@gmail.com>
> > >> > > > > > > > wrote:
> > >> > > > > > > >
> > >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> > >> follow-up KIPs
> > >> > > > sounds
> > >> > > > > > > > like
> > >> > > > > > > > > a good idea.
> > >> > > > > > > > >
> > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > >> ismael@juma.me.uk>
> > >> > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > > Hi Boyang,
> > >> > > > > > > > > >
> > >> > > > > > > > > > Yes, there will be several KIPs that will discuss the
> > >> items you
> > >> > > > > > > > describe
> > >> > > > > > > > > in
> > >> > > > > > > > > > detail. Colin, it may be helpful to make this clear in
> > >> the KIP
> > >> > > > 500
> > >> > > > > > > > > > description.
> > >> > > > > > > > > >
> > >> > > > > > > > > > Ismael
> > >> > > > > > > > > >
> > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > >> > > > reluctanthero104@gmail.com
> > >> > > > > > > > >
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > > > Thanks Colin for initiating this important effort!
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > One question I have is whether we have a session
> > >> discussing
> > >> > > > the
> > >> > > > > > > > > > controller
> > >> > > > > > > > > > > failover in the new architecture? I know we are using
> > >> Raft
> > >> > > > protocol
> > >> > > > > > > > to
> > >> > > > > > > > > > > failover, yet it's still valuable to discuss the
> > >> steps new
> > >> > > > cluster is
> > >> > > > > > > > > > going
> > >> > > > > > > > > > > to take to reach the stable stage again, so that we
> > >> could
> > >> > > > easily
> > >> > > > > > > > > measure
> > >> > > > > > > > > > > the availability of the metadata servers.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Another suggestion I have is to write a step-by-step
> > >> design
> > >> > > > doc like
> > >> > > > > > > > > what
> > >> > > > > > > > > > > we did in KIP-98
> > >> > > > > > > > > > > <
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > >
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >> > > > > > > > > > > >,
> > >> > > > > > > > > > > including the new request protocols and how they are
> > >> > > > interacting in
> > >> > > > > > > > the
> > >> > > > > > > > > > new
> > >> > > > > > > > > > > cluster. For a complicated change like this, an
> > >> > > > implementation design
> > >> > > > > > > > > doc
> > >> > > > > > > > > > > help a lot in the review process, otherwise most
> > >> discussions
> > >> > > > we have
> > >> > > > > > > > > will
> > >> > > > > > > > > > > focus on high level and lose important details as we
> > >> > > > discover them in
> > >> > > > > > > > > the
> > >> > > > > > > > > > > post-agreement phase.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Boyang
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > >> > > > cmccabe@apache.org>
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia
> > >> Sancio
> > >> > > > wrote:
> > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
> > >> comments
> > >> > > > and
> > >> > > > > > > > > questions.
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> > >> mentioned the
> > >> > > > > > > > LeaderAndIsr
> > >> > > > > > > > > > and
> > >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates which
> > >> the
> > >> > > > controller
> > >> > > > > > > > > > pushes,
> > >> > > > > > > > > > > > such
> > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is
> > >> your
> > >> > > > thinking
> > >> > > > > > > > that
> > >> > > > > > > > > > we
> > >> > > > > > > > > > > > will
> > >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > >> > > > UpdateMetadata only
> > >> > > > > > > > and
> > >> > > > > > > > > > add
> > >> > > > > > > > > > > > > topic configuration in this state?
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Hi Jose,
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Thanks for taking a look.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to replace both
> > >> > > > > > > > > > LeaderAndIsrRequest
> > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
> > >> would be
> > >> > > > fetched
> > >> > > > > > > > > along
> > >> > > > > > > > > > > > with the other metadata.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > In the section "Broker Metadata Management", you
> > >> mention
> > >> > > > "Just
> > >> > > > > > > > like
> > >> > > > > > > > > > > with
> > >> > > > > > > > > > > > a
> > >> > > > > > > > > > > > > fetch request, the broker will track the offset
> > >> of the
> > >> > > > last
> > >> > > > > > > > updates
> > >> > > > > > > > > > it
> > >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> > >> requires that
> > >> > > > the
> > >> > > > > > > > > followers
> > >> > > > > > > > > > > > keep
> > >> > > > > > > > > > > > > all of the log entries (term/epoch and offset)
> > >> that are
> > >> > > > after the
> > >> > > > > > > > > > > > > highwatermark. Any log entry before the
> > >> highwatermark
> > >> > > > can be
> > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > >> MetadataFetch API
> > >> > > > to only
> > >> > > > > > > > > return
> > >> > > > > > > > > > > log
> > >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> > >> > > > replication API
> > >> > > > > > > > > > which
> > >> > > > > > > > > > > > > will replicate/fetch log entries after the
> > >> highwatermark
> > >> > > > for
> > >> > > > > > > > > > consensus?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> > >> metadata
> > >> > > > updates to
> > >> > > > > > > > the
> > >> > > > > > > > > > > > brokers until they've been stored on a majority of
> > >> the
> > >> > > > Raft nodes.
> > >> > > > > > > > > The
> > >> > > > > > > > > > > > most obvious way to do that, like you mentioned, is
> > >> to
> > >> > > > have the
> > >> > > > > > > > > brokers
> > >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
> > >> might be
> > >> > > > a more
> > >> > > > > > > > > clever
> > >> > > > > > > > > > > way
> > >> > > > > > > > > > > > to do it by fetching the data, but not having the
> > >> brokers
> > >> > > > act on it
> > >> > > > > > > > > > until
> > >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth it
> > >> or
> > >> > > > not.  We'll
> > >> > > > > > > > > > discuss
> > >> > > > > > > > > > > > this more in a separate KIP that just discusses
> > >> just Raft.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > >> mention "the
> > >> > > > > > > > > controller
> > >> > > > > > > > > > > will
> > >> > > > > > > > > > > > > send a full metadata image rather than a series of
> > >> > > > deltas". This
> > >> > > > > > > > > KIP
> > >> > > > > > > > > > > > > doesn't go into the set of operations that need
> > >> to be
> > >> > > > supported
> > >> > > > > > > > on
> > >> > > > > > > > > > top
> > >> > > > > > > > > > > of
> > >> > > > > > > > > > > > > Raft but it would be interested if this "full
> > >> metadata
> > >> > > > image"
> > >> > > > > > > > could
> > >> > > > > > > > > > be
> > >> > > > > > > > > > > > > express also as deltas. For example, assuming we
> > >> are
> > >> > > > replicating
> > >> > > > > > > > a
> > >> > > > > > > > > > map
> > >> > > > > > > > > > > > this
> > >> > > > > > > > > > > > > "full metadata image" could be a sequence of "put"
> > >> > > > operations
> > >> > > > > > > > > (znode
> > >> > > > > > > > > > > > create
> > >> > > > > > > > > > > > > to borrow ZK semantics).
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > The full image can definitely be expressed as a sum
> > >> of
> > >> > > > deltas.  At
> > >> > > > > > > > > some
> > >> > > > > > > > > > > > point, the number of deltas will get large enough
> > >> that
> > >> > > > sending a
> > >> > > > > > > > full
> > >> > > > > > > > > > > image
> > >> > > > > > > > > > > > is better, though.  One question that we're still
> > >> thinking
> > >> > > > about is
> > >> > > > > > > > > how
> > >> > > > > > > > > > > > much of this can be shared with generic Kafka log
> > >> code,
> > >> > > > and how
> > >> > > > > > > > much
> > >> > > > > > > > > > > should
> > >> > > > > > > > > > > > be different.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > >> mention
> > >> > > > "This
> > >> > > > > > > > request
> > >> > > > > > > > > > will
> > >> > > > > > > > > > > > > double as a heartbeat, letting the controller
> > >> know that
> > >> > > > the
> > >> > > > > > > > broker
> > >> > > > > > > > > is
> > >> > > > > > > > > > > > > alive". In section "Broker State Machine", you
> > >> mention
> > >> > > > "The
> > >> > > > > > > > > > > MetadataFetch
> > >> > > > > > > > > > > > > API serves as this registration mechanism". Does
> > >> this
> > >> > > > mean that
> > >> > > > > > > > the
> > >> > > > > > > > > > > > > MetadataFetch Request will optionally include
> > >> broker
> > >> > > > > > > > configuration
> > >> > > > > > > > > > > > > information?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > I was originally thinking that the
> > >> MetadataFetchRequest
> > >> > > > should
> > >> > > > > > > > > include
> > >> > > > > > > > > > > > broker configuration information.  Thinking about
> > >> this
> > >> > > > more, maybe
> > >> > > > > > > > we
> > >> > > > > > > > > > > > should just have a special registration RPC that
> > >> contains
> > >> > > > that
> > >> > > > > > > > > > > information,
> > >> > > > > > > > > > > > to avoid sending it over the wire all the time.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > Does this also mean that MetadataFetch request
> > >> will
> > >> > > > result in
> > >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > >> replication
> > >> > > > protocol
> > >> > > > > > > > > before
> > >> > > > > > > > > > > you
> > >> > > > > > > > > > > > > can send the associated MetadataFetch Response?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > I think we should require the broker to be out of
> > >> the
> > >> > > > Offline state
> > >> > > > > > > > > > > before
> > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> > >> > > > registration
> > >> > > > > > > > RPC
> > >> > > > > > > > > > > > should have completed first.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Broker State", you mention that a
> > >> broker can
> > >> > > > > > > > transition
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > > online after it is caught with the metadata. What
> > >> do you
> > >> > > > mean by
> > >> > > > > > > > > > this?
> > >> > > > > > > > > > > > > Metadata is always changing. How does the broker
> > >> know
> > >> > > > that it is
> > >> > > > > > > > > > caught
> > >> > > > > > > > > > > > up
> > >> > > > > > > > > > > > > since it doesn't participate in the consensus or
> > >> the
> > >> > > > advancement
> > >> > > > > > > > of
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > highwatermark?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > That's a good point.  Being "caught up" is somewhat
> > >> of a
> > >> > > > fuzzy
> > >> > > > > > > > > concept
> > >> > > > > > > > > > > > here, since the brokers do not participate in the
> > >> metadata
> > >> > > > > > > > consensus.
> > >> > > > > > > > > > I
> > >> > > > > > > > > > > > think ideally we would want to define it in terms
> > >> of time
> > >> > > > ("the
> > >> > > > > > > > > broker
> > >> > > > > > > > > > > has
> > >> > > > > > > > > > > > all the updates from the last 2 minutes", for
> > >> example.)
> > >> > > > We should
> > >> > > > > > > > > > spell
> > >> > > > > > > > > > > > this out better in the KIP.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Start the controller quorum nodes",
> > >> you
> > >> > > > mention "Once
> > >> > > > > > > > > it
> > >> > > > > > > > > > > has
> > >> > > > > > > > > > > > > taken over the /controller node, the active
> > >> controller
> > >> > > > will
> > >> > > > > > > > proceed
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > load
> > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will write out
> > >> this
> > >> > > > information
> > >> > > > > > > > to
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > quorum's metadata storage.  After this point, the
> > >> > > > metadata quorum
> > >> > > > > > > > > > will
> > >> > > > > > > > > > > be
> > >> > > > > > > > > > > > > the metadata store of record, rather than the
> > >> data in
> > >> > > > ZooKeeper."
> > >> > > > > > > > > > > During
> > >> > > > > > > > > > > > > this migration do should we expect to have a
> > >> small period
> > >> > > > > > > > > controller
> > >> > > > > > > > > > > > > unavailability while the controller replicas this
> > >> state
> > >> > > > to all of
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > raft
> > >> > > > > > > > > > > > > nodes in the controller quorum and we buffer new
> > >> > > > controller API
> > >> > > > > > > > > > > requests?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Yes, the controller would be unavailable during this
> > >> > > > time.  I don't
> > >> > > > > > > > > > think
> > >> > > > > > > > > > > > this will be that different from the current period
> > >> of
> > >> > > > > > > > unavailability
> > >> > > > > > > > > > > when
> > >> > > > > > > > > > > > a new controller starts up and needs to load the
> > >> full
> > >> > > > state from
> > >> > > > > > > > ZK.
> > >> > > > > > > > > > The
> > >> > > > > > > > > > > > main difference is that in this period, we'd have
> > >> to write
> > >> > > > to the
> > >> > > > > > > > > > > > controller quorum rather than just to memory.  But
> > >> we
> > >> > > > believe this
> > >> > > > > > > > > > should
> > >> > > > > > > > > > > > be pretty fast.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > regards,
> > >> > > > > > > > > > > > Colin
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > Thanks!
> > >> > > > > > > > > > > > > -Jose
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> > >
> > >> > > --
> > >> > > David Arthur
> > >> > >
> > >> >
> > >>
> > >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Eno Thereska <en...@gmail.com>.
Hi Colin,

Nice KIP! For such a big change it would be good to add a pointer or
two to related work that provides some sort of soft proof that the
approach taken makes sense. Also such work often builds on other work
and it might be useful to trace its roots. May I recommend adding a
pointer to "Tango: Distributed Data Structures over a Shared Log"
(http://www.cs.cornell.edu/~taozou/sosp13/tangososp.pdf)? There are
other papers that are related (e.g., a more recent one one "The
FuzzyLog: A Partially Ordered Shared Log"
(https://www.usenix.org/system/files/osdi18-lockerman.pdf)).

Both papers would add to the strength of your motivation.

Cheers
Eno

On Wed, Aug 21, 2019 at 12:22 PM Ron Dagostino <rn...@gmail.com> wrote:
>
> Hi Colin.  I like the concept of a "bridge release" for migrating off of
> Zookeeper, but I worry that it may become a bottleneck if people hesitate
> to replace Zookeeper -- they would be unable to adopt newer versions of
> Kafka until taking (what feels to them like) a giant leap.  As an example,
> assuming version 4.0.x of Kafka is the supported bridge release, I  would
> not be surprised if uptake of the 4.x release and the time-based releases
> that follow it end up being much slower due to the perceived barrier.
>
> Any perceived barrier could be lowered if the 4.0.x release could
> optionally continue to use Zookeeper -- then the cutover would be two
> incremental steps (move to 4.0.x, then replace Zookeeper while staying on
> 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> Zookeeper in one fell swoop).
>
> Regardless of whether what I wrote above has merit or not, I think the KIP
> should be more explicit about what the upgrade constraints actually are.
> Can the bridge release be adopted with Zookeeper remaining in place and
> then cutting over as a second, follow-on step, or must the Controller
> Quorum nodes be started first and the bridge release cannot be used with
> Zookeeper at all?  If the bridge release cannot be used with Zookeeper at
> all, then no version at or beyond the bridge release is available
> unless/until abandoning Zookeeper; if the bridge release can be used with
> Zookeeper, then is it the only version that can be used with Zookeeper, or
> can Zookeeper be kept for additional releases if desired?
>
> Ron
>
> On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com> wrote:
>
> > Hi Colin.  The diagram up at the top confused me -- specifically, the
> > lines connecting the controller/active-controller to the brokers.  I had
> > assumed the arrows on those lines represented the direction of data flow,
> > but that is not the case; the arrows actually identify the target of the
> > action, and the non-arrowed end indicates the initiator of the action.  For
> > example, the lines point from the controller to the brokers in the "today"
> > section on the left to show that the controller pushes to the brokers; the
> > lines point from the brokers to the active-controller in the "tomorrow"
> > section on the right to show that the brokers pull from the
> > active-controller.  As I said, this confused me because my gut instinct was
> > to interpret the arrow as indicating the direction of data flow, and when I
> > look at the "tomorrow" picture on the right I initially thought information
> > was moving from the brokers to the active-controller.  Did you consider
> > drawing that picture with the arrows reversed in the "tomorrow" side so
> > that the arrows represent the direction of data flow, and then add the
> > labels "push" on the "today" side and "pull" on the "tomorrow" side to
> > indicate who initiates the data flow?  It occurs to me that this picture
> > may end up being widely distributed, so it might be in everyone's interest
> > to proactively avoid any possible confusion by being more explicit.
> >
> > Minor corrections?
> > <<<In the current world, a broker which can contact ZooKeeper but which
> > is partitioned from the active controller
> > >>>In the current world, a broker which can contact ZooKeeper but which
> > is partitioned from the controller
> >
> > <<<Eventually, the controller will ask the broker to finally go offline
> > >>>Eventually, the active controller will ask the broker to finally go
> > offline
> >
> > <<<New versions of the clients should send these operations directly to
> > the controller
> > >>>New versions of the clients should send these operations directly to
> > the active controller
> >
> > <<<In the post-ZK world, the leader will make an RPC to the controller
> > instead
> > >>>In the post-ZK world, the leader will make an RPC to the active
> > controller instead
> >
> > <<<For example, the brokers may need to forward their requests to the
> > controller.
> > >>>For example, the brokers may need to forward their requests to the
> > active controller.
> >
> > <<<The new controller will monitor ZooKeeper for legacy broker node
> > registrations
> > >>>The new (active) controller will monitor ZooKeeper for legacy broker
> > node registrations
> >
> > Ron
> >
> > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org> wrote:
> >
> >> Hi all,
> >>
> >> The KIP has been out for a while, so I'm thinking about calling a vote
> >> some time this week.
> >>
> >> best,
> >> Colin
> >>
> >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> >> > > Thanks for the KIP, Colin. This looks great!
> >> > >
> >> > > I really like the idea of separating the Controller and Broker JVMs.
> >> > >
> >> > > As you alluded to above, it might be nice to have a separate
> >> > > broker-registration API to avoid overloading the metadata fetch API.
> >> > >
> >> >
> >> > Hi David,
> >> >
> >> > Thanks for taking a look.
> >> >
> >> > I removed the sentence about MetadataFetch also serving as the broker
> >> > registration API.  I think I agree that we will probably want a
> >> > separate RPC to fill this role.  We will have a follow-on KIP that will
> >> > go into more detail about metadata propagation and registration in the
> >> > post-ZK world.  That KIP will also have a full description of the
> >> > registration RPC, etc.  For now, I think the important part for KIP-500
> >> > is that the broker registers with the controller quorum.  On
> >> > registration, the controller quorum assigns it a new broker epoch,
> >> > which can distinguish successive broker incarnations.
> >> >
> >> > >
> >> > > When a broker gets a metadata delta, will it be a sequence of deltas
> >> since
> >> > > the last update or a cumulative delta since the last update?
> >> > >
> >> >
> >> > It will be a sequence of deltas.  Basically, the broker will be reading
> >> > from the metadata log.
> >> >
> >> > >
> >> > > Will we include any kind of integrity check on the deltas to ensure
> >> the brokers
> >> > > have applied them correctly? Perhaps this will be addressed in one of
> >> the
> >> > > follow-on KIPs.
> >> > >
> >> >
> >> > In general, we will have checksums on the metadata that we fetch.  This
> >> > is similar to how we have checksums on regular data.  Or if the
> >> > question is about catching logic errors in the metadata handling code,
> >> > that sounds more like something that should be caught by test cases.
> >> >
> >> > best,
> >> > Colin
> >> >
> >> >
> >> > > Thanks!
> >> > >
> >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> > >
> >> > > > Hi Mickael,
> >> > > >
> >> > > > Thanks for taking a look.
> >> > > >
> >> > > > I don't think we want to support that kind of multi-tenancy at the
> >> > > > controller level.  If the cluster is small enough that we want to
> >> pack the
> >> > > > controller(s) with something else, we could run them alongside the
> >> brokers,
> >> > > > or possibly inside three of the broker JVMs.
> >> > > >
> >> > > > best,
> >> > > > Colin
> >> > > >
> >> > > >
> >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> >> > > > > Thank Colin for kickstarting this initiative.
> >> > > > >
> >> > > > > Just one question.
> >> > > > > - A nice feature of Zookeeper is the ability to use chroots and
> >> have
> >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is this
> >> > > > > something we should keep?
> >> > > > >
> >> > > > > Thanks
> >> > > > >
> >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> > > > > >
> >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> >> > > > > > > Hi Colin,
> >> > > > > > >
> >> > > > > > > Thanks for the KIP.
> >> > > > > > >
> >> > > > > > > Currently ZooKeeper provides a convenient notification
> >> mechanism for
> >> > > > > > > knowing that broker and topic configuration has changed. While
> >> > > > KIP-500 does
> >> > > > > > > suggest that incremental metadata update is expected to come
> >> to
> >> > > > clients
> >> > > > > > > eventually, that would seem to imply that for some number of
> >> > > > releases there
> >> > > > > > > would be no equivalent mechanism for knowing about config
> >> changes.
> >> > > > Is there
> >> > > > > > > any thinking at this point about how a similar notification
> >> might be
> >> > > > > > > provided in the future?
> >> > > > > >
> >> > > > > > We could eventually have some inotify-like mechanism where
> >> clients
> >> > > > could register interest in various types of events and got notified
> >> when
> >> > > > they happened.  Reading the metadata log is conceptually simple.
> >> The main
> >> > > > complexity would be in setting up an API that made sense and that
> >> didn't
> >> > > > unduly constrain future implementations.  We'd have to think
> >> carefully
> >> > > > about what the real use-cases for this were, though.
> >> > > > > >
> >> > > > > > best,
> >> > > > > > Colin
> >> > > > > >
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > >
> >> > > > > > > Tom
> >> > > > > > >
> >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> >> > > > viktorsomogyi@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hey Colin,
> >> > > > > > > >
> >> > > > > > > > I think this is a long-awaited KIP, thanks for driving it.
> >> I'm
> >> > > > excited to
> >> > > > > > > > see this in Kafka once. I collected my questions (and I
> >> accept the
> >> > > > "TBD"
> >> > > > > > > > answer as they might be a bit deep for this high level :) ).
> >> > > > > > > > 1.) Are there any specific reasons for the Controller just
> >> > > > periodically
> >> > > > > > > > persisting its state on disk periodically instead of
> >> > > > asynchronously with
> >> > > > > > > > every update? Wouldn't less frequent saves increase the
> >> chance for
> >> > > > missing
> >> > > > > > > > a state change if the controller crashes between two saves?
> >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from the
> >> follower
> >> > > > > > > > controllers? I assume that followers would have up-to-date
> >> > > > information
> >> > > > > > > > therefore brokers could fetch from there in theory.
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > > Viktor
> >> > > > > > > >
> >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> >> > > > reluctanthero104@gmail.com>
> >> > > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> >> follow-up KIPs
> >> > > > sounds
> >> > > > > > > > like
> >> > > > > > > > > a good idea.
> >> > > > > > > > >
> >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> >> ismael@juma.me.uk>
> >> > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Hi Boyang,
> >> > > > > > > > > >
> >> > > > > > > > > > Yes, there will be several KIPs that will discuss the
> >> items you
> >> > > > > > > > describe
> >> > > > > > > > > in
> >> > > > > > > > > > detail. Colin, it may be helpful to make this clear in
> >> the KIP
> >> > > > 500
> >> > > > > > > > > > description.
> >> > > > > > > > > >
> >> > > > > > > > > > Ismael
> >> > > > > > > > > >
> >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> >> > > > reluctanthero104@gmail.com
> >> > > > > > > > >
> >> > > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > Thanks Colin for initiating this important effort!
> >> > > > > > > > > > >
> >> > > > > > > > > > > One question I have is whether we have a session
> >> discussing
> >> > > > the
> >> > > > > > > > > > controller
> >> > > > > > > > > > > failover in the new architecture? I know we are using
> >> Raft
> >> > > > protocol
> >> > > > > > > > to
> >> > > > > > > > > > > failover, yet it's still valuable to discuss the
> >> steps new
> >> > > > cluster is
> >> > > > > > > > > > going
> >> > > > > > > > > > > to take to reach the stable stage again, so that we
> >> could
> >> > > > easily
> >> > > > > > > > > measure
> >> > > > > > > > > > > the availability of the metadata servers.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Another suggestion I have is to write a step-by-step
> >> design
> >> > > > doc like
> >> > > > > > > > > what
> >> > > > > > > > > > > we did in KIP-98
> >> > > > > > > > > > > <
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > > > > > > > > > >,
> >> > > > > > > > > > > including the new request protocols and how they are
> >> > > > interacting in
> >> > > > > > > > the
> >> > > > > > > > > > new
> >> > > > > > > > > > > cluster. For a complicated change like this, an
> >> > > > implementation design
> >> > > > > > > > > doc
> >> > > > > > > > > > > help a lot in the review process, otherwise most
> >> discussions
> >> > > > we have
> >> > > > > > > > > will
> >> > > > > > > > > > > focus on high level and lose important details as we
> >> > > > discover them in
> >> > > > > > > > > the
> >> > > > > > > > > > > post-agreement phase.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Boyang
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> >> > > > cmccabe@apache.org>
> >> > > > > > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia
> >> Sancio
> >> > > > wrote:
> >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
> >> comments
> >> > > > and
> >> > > > > > > > > questions.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> >> mentioned the
> >> > > > > > > > LeaderAndIsr
> >> > > > > > > > > > and
> >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates which
> >> the
> >> > > > controller
> >> > > > > > > > > > pushes,
> >> > > > > > > > > > > > such
> >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is
> >> your
> >> > > > thinking
> >> > > > > > > > that
> >> > > > > > > > > > we
> >> > > > > > > > > > > > will
> >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> >> > > > UpdateMetadata only
> >> > > > > > > > and
> >> > > > > > > > > > add
> >> > > > > > > > > > > > > topic configuration in this state?
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Hi Jose,
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Thanks for taking a look.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > The goal is for MetadataFetchRequest to replace both
> >> > > > > > > > > > LeaderAndIsrRequest
> >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
> >> would be
> >> > > > fetched
> >> > > > > > > > > along
> >> > > > > > > > > > > > with the other metadata.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > In the section "Broker Metadata Management", you
> >> mention
> >> > > > "Just
> >> > > > > > > > like
> >> > > > > > > > > > > with
> >> > > > > > > > > > > > a
> >> > > > > > > > > > > > > fetch request, the broker will track the offset
> >> of the
> >> > > > last
> >> > > > > > > > updates
> >> > > > > > > > > > it
> >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> >> requires that
> >> > > > the
> >> > > > > > > > > followers
> >> > > > > > > > > > > > keep
> >> > > > > > > > > > > > > all of the log entries (term/epoch and offset)
> >> that are
> >> > > > after the
> >> > > > > > > > > > > > > highwatermark. Any log entry before the
> >> highwatermark
> >> > > > can be
> >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> >> MetadataFetch API
> >> > > > to only
> >> > > > > > > > > return
> >> > > > > > > > > > > log
> >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> >> > > > replication API
> >> > > > > > > > > > which
> >> > > > > > > > > > > > > will replicate/fetch log entries after the
> >> highwatermark
> >> > > > for
> >> > > > > > > > > > consensus?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> >> metadata
> >> > > > updates to
> >> > > > > > > > the
> >> > > > > > > > > > > > brokers until they've been stored on a majority of
> >> the
> >> > > > Raft nodes.
> >> > > > > > > > > The
> >> > > > > > > > > > > > most obvious way to do that, like you mentioned, is
> >> to
> >> > > > have the
> >> > > > > > > > > brokers
> >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
> >> might be
> >> > > > a more
> >> > > > > > > > > clever
> >> > > > > > > > > > > way
> >> > > > > > > > > > > > to do it by fetching the data, but not having the
> >> brokers
> >> > > > act on it
> >> > > > > > > > > > until
> >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth it
> >> or
> >> > > > not.  We'll
> >> > > > > > > > > > discuss
> >> > > > > > > > > > > > this more in a separate KIP that just discusses
> >> just Raft.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> >> mention "the
> >> > > > > > > > > controller
> >> > > > > > > > > > > will
> >> > > > > > > > > > > > > send a full metadata image rather than a series of
> >> > > > deltas". This
> >> > > > > > > > > KIP
> >> > > > > > > > > > > > > doesn't go into the set of operations that need
> >> to be
> >> > > > supported
> >> > > > > > > > on
> >> > > > > > > > > > top
> >> > > > > > > > > > > of
> >> > > > > > > > > > > > > Raft but it would be interested if this "full
> >> metadata
> >> > > > image"
> >> > > > > > > > could
> >> > > > > > > > > > be
> >> > > > > > > > > > > > > express also as deltas. For example, assuming we
> >> are
> >> > > > replicating
> >> > > > > > > > a
> >> > > > > > > > > > map
> >> > > > > > > > > > > > this
> >> > > > > > > > > > > > > "full metadata image" could be a sequence of "put"
> >> > > > operations
> >> > > > > > > > > (znode
> >> > > > > > > > > > > > create
> >> > > > > > > > > > > > > to borrow ZK semantics).
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > The full image can definitely be expressed as a sum
> >> of
> >> > > > deltas.  At
> >> > > > > > > > > some
> >> > > > > > > > > > > > point, the number of deltas will get large enough
> >> that
> >> > > > sending a
> >> > > > > > > > full
> >> > > > > > > > > > > image
> >> > > > > > > > > > > > is better, though.  One question that we're still
> >> thinking
> >> > > > about is
> >> > > > > > > > > how
> >> > > > > > > > > > > > much of this can be shared with generic Kafka log
> >> code,
> >> > > > and how
> >> > > > > > > > much
> >> > > > > > > > > > > should
> >> > > > > > > > > > > > be different.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> >> mention
> >> > > > "This
> >> > > > > > > > request
> >> > > > > > > > > > will
> >> > > > > > > > > > > > > double as a heartbeat, letting the controller
> >> know that
> >> > > > the
> >> > > > > > > > broker
> >> > > > > > > > > is
> >> > > > > > > > > > > > > alive". In section "Broker State Machine", you
> >> mention
> >> > > > "The
> >> > > > > > > > > > > MetadataFetch
> >> > > > > > > > > > > > > API serves as this registration mechanism". Does
> >> this
> >> > > > mean that
> >> > > > > > > > the
> >> > > > > > > > > > > > > MetadataFetch Request will optionally include
> >> broker
> >> > > > > > > > configuration
> >> > > > > > > > > > > > > information?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > I was originally thinking that the
> >> MetadataFetchRequest
> >> > > > should
> >> > > > > > > > > include
> >> > > > > > > > > > > > broker configuration information.  Thinking about
> >> this
> >> > > > more, maybe
> >> > > > > > > > we
> >> > > > > > > > > > > > should just have a special registration RPC that
> >> contains
> >> > > > that
> >> > > > > > > > > > > information,
> >> > > > > > > > > > > > to avoid sending it over the wire all the time.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > Does this also mean that MetadataFetch request
> >> will
> >> > > > result in
> >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> >> replication
> >> > > > protocol
> >> > > > > > > > > before
> >> > > > > > > > > > > you
> >> > > > > > > > > > > > > can send the associated MetadataFetch Response?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > I think we should require the broker to be out of
> >> the
> >> > > > Offline state
> >> > > > > > > > > > > before
> >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> >> > > > registration
> >> > > > > > > > RPC
> >> > > > > > > > > > > > should have completed first.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Broker State", you mention that a
> >> broker can
> >> > > > > > > > transition
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > online after it is caught with the metadata. What
> >> do you
> >> > > > mean by
> >> > > > > > > > > > this?
> >> > > > > > > > > > > > > Metadata is always changing. How does the broker
> >> know
> >> > > > that it is
> >> > > > > > > > > > caught
> >> > > > > > > > > > > > up
> >> > > > > > > > > > > > > since it doesn't participate in the consensus or
> >> the
> >> > > > advancement
> >> > > > > > > > of
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > highwatermark?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > That's a good point.  Being "caught up" is somewhat
> >> of a
> >> > > > fuzzy
> >> > > > > > > > > concept
> >> > > > > > > > > > > > here, since the brokers do not participate in the
> >> metadata
> >> > > > > > > > consensus.
> >> > > > > > > > > > I
> >> > > > > > > > > > > > think ideally we would want to define it in terms
> >> of time
> >> > > > ("the
> >> > > > > > > > > broker
> >> > > > > > > > > > > has
> >> > > > > > > > > > > > all the updates from the last 2 minutes", for
> >> example.)
> >> > > > We should
> >> > > > > > > > > > spell
> >> > > > > > > > > > > > this out better in the KIP.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Start the controller quorum nodes",
> >> you
> >> > > > mention "Once
> >> > > > > > > > > it
> >> > > > > > > > > > > has
> >> > > > > > > > > > > > > taken over the /controller node, the active
> >> controller
> >> > > > will
> >> > > > > > > > proceed
> >> > > > > > > > > > to
> >> > > > > > > > > > > > load
> >> > > > > > > > > > > > > the full state of ZooKeeper.  It will write out
> >> this
> >> > > > information
> >> > > > > > > > to
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > quorum's metadata storage.  After this point, the
> >> > > > metadata quorum
> >> > > > > > > > > > will
> >> > > > > > > > > > > be
> >> > > > > > > > > > > > > the metadata store of record, rather than the
> >> data in
> >> > > > ZooKeeper."
> >> > > > > > > > > > > During
> >> > > > > > > > > > > > > this migration do should we expect to have a
> >> small period
> >> > > > > > > > > controller
> >> > > > > > > > > > > > > unavailability while the controller replicas this
> >> state
> >> > > > to all of
> >> > > > > > > > > the
> >> > > > > > > > > > > > raft
> >> > > > > > > > > > > > > nodes in the controller quorum and we buffer new
> >> > > > controller API
> >> > > > > > > > > > > requests?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Yes, the controller would be unavailable during this
> >> > > > time.  I don't
> >> > > > > > > > > > think
> >> > > > > > > > > > > > this will be that different from the current period
> >> of
> >> > > > > > > > unavailability
> >> > > > > > > > > > > when
> >> > > > > > > > > > > > a new controller starts up and needs to load the
> >> full
> >> > > > state from
> >> > > > > > > > ZK.
> >> > > > > > > > > > The
> >> > > > > > > > > > > > main difference is that in this period, we'd have
> >> to write
> >> > > > to the
> >> > > > > > > > > > > > controller quorum rather than just to memory.  But
> >> we
> >> > > > believe this
> >> > > > > > > > > > should
> >> > > > > > > > > > > > be pretty fast.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > regards,
> >> > > > > > > > > > > > Colin
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Thanks!
> >> > > > > > > > > > > > > -Jose
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> > >
> >> > > --
> >> > > David Arthur
> >> > >
> >> >
> >>
> >

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ron Dagostino <rn...@gmail.com>.
Thanks, Colin.  That all makes sense, especially the part about the onerous
testing requirements associated with supporting both Zookeeper and the new
metadata quorum simultaneously.  Given that, I now buy into the idea that
the transition to the new metadata quorum becomes the main path forward
once the final bridge release is cut.  There may be back-ports of bugs and
features based on demand as you suggest at the end of your reply, but there
is no guarantee of that happening.  I'm good with that now.

Ron

On Tue, Sep 3, 2019 at 1:05 PM Colin McCabe <cm...@apache.org> wrote:

> On Mon, Sep 2, 2019, at 07:51, Ron Dagostino wrote:
> > Hi Colin.  It is not unusual for customers to wait before upgrading —
> > to avoid so-called “point-zero” releases — to avoid as many of the
> > inevitable bugs that ride along with new functionality as possible.
> > Removal of Zookeeper is going feel especially risky to these customers,
> > and arguably it is going to feel risky even to customers who might
> > otherwise be less sensitive to upgrade risk.
> >
> > This leads me to believe it is reasonable to expect that the uptake of
> > the new ZK-less consensus quorum could be delayed in many installations
> > — that such customers might wait longer than usual to adopt the feature
> > and abandon their Zookeeper servers.
> >
> > Will it be possible to use releases beyond the bridge release and not
> > abandon Zookeeper?  For example, what would happen if post-bridge the
> > new consensus quorum servers are never started?  Would Kafka still work
> > fine?  At what point MUST Zookeeper be abandoned?  Taking the
> > perspective of the above customers, I think they would prefer to have
> > others adopt the new ZK-less consensus quorum for several months and
> > encounter many of the inevitable bugs before adopting it themselves.
> > But at the same time they will not want to be stuck on the bridge
> > release that whole time because there are going to be both bug fixes
> > and new features that they will want to take advantage of.
> >
> > If the bridge release is the last one that supports Zookeeper, and if
> > some customers stay on that release for a while, then I could see those
> > customers wanting back-ports of bug fixes and features to occur for a
> > period of time that extends beyond what is normally done.
> >
> > Basically, to sum all of the above up, I think there is a reasonable
> > probability that a single bridge release only could become a potential
> > barrier that causes angst for the project and the community.
> >
> > I wonder if it would be in the interest of the project and the
> > community to mitigate the risk of there being a bridge release barrier
> > by extending the time when ZK would still be supported — perhaps for up
> > to a year — and the new co send us quorum could remain optional.
> >
> > Ron
> >
>
> Hi Ron,
>
> Changing things always involves risk.  This is why we are trying to do as
> much as we can incrementally.  For example, removing ZK dependencies from
> tools, and from brokers.  However, there are things that can't really be
> done incrementally, and one of these is switching over to a new metadata
> store.
>
> It might seem like supporting multiple options for where to store metadata
> would be safer somehow, but actually this is not the case.  Having to
> support totally different code paths involves a lot more work and a lot
> more testing.  We already have configurations that aren't tested enough.
> Doubling (at least) the number of configurations we have to test is a
> non-starter.
>
> This also ties in with the discussion in the KIP about why we don't plan
> on supporting pluggable consensus or pluggable metadata storage.  Doing
> this would force us to use only the least-common denominator features of
> every metadata storage.  We would not be able to take advantage of metadata
> as a stream of events, or any of the features that ZK doesn't have.
> Configuration would also be quite complex.
>
> As the KIP states, the upgrade from a bridge release (there may be several
> bridge releases) to ZK will have no impact on clients.  It also won't have
> any impact on cluster sizing (ZK nodes will simply become controller
> nodes).  And it will be possible to do with a rolling upgrade.  I agree
> that some people may be nervous about running the new software, and we may
> want to have more point releases of the older branches.
>
> This is something that we'll discuss when people propose release
> schedules.  In general, this isn't fundamentally different than someone
> wanting a new release of 1.x because they don't want to upgrade to 2.x.  If
> there's enough interest, we'll do it.
>
> best,
> Colin
>
> >
> > > On Aug 26, 2019, at 6:55 PM, Colin McCabe <cm...@apache.org> wrote:
> > >
> > > Hi Ryanne,
> > >
> > > Good point.  I added a section titled "future work" with information
> about the follow-on KIPs that we discussed here.
> > >
> > > best,
> > > Colin
> > >
> > >
> > >> On Fri, Aug 23, 2019, at 13:15, Ryanne Dolan wrote:
> > >> Thanks Colin, sgtm. Please make this clear in the KIP -- otherwise it
> is
> > >> hard to nail down what we are voting for.
> > >>
> > >> Ryanne
> > >>
> > >>
> > >>> On Fri, Aug 23, 2019, 12:58 PM Colin McCabe <cm...@apache.org>
> wrote:
> > >>>
> > >>>> On Fri, Aug 23, 2019, at 06:24, Ryanne Dolan wrote:
> > >>>> Colin, can you outline what specifically would be in scope for this
> KIP
> > >>> vs
> > >>>> deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
> > >>>> section? Is the idea to get to the bridge release with this KIP,
> and then
> > >>>> go from there?
> > >>>>
> > >>>> Ryanne
> > >>>>
> > >>>
> > >>> Hi Ryanne,
> > >>>
> > >>> The goal for KIP-500 is to set out an overall vision for how we will
> > >>> remove ZooKeeper and transition to managing metadata via a controller
> > >>> quorum.
> > >>>
> > >>> We will create follow-on KIPs that will lay out the specific details
> of
> > >>> each step.
> > >>>
> > >>> * A KIP for allowing kafka-configs.sh to change topic configurations
> > >>> without using ZooKeeper.  (It can already change broker
> configurations
> > >>> without ZK)
> > >>>
> > >>> * A KIP for adding APIs to replace direct ZK access by the brokers.
> > >>>
> > >>> * A KIP to describe Raft replication in Kafka, including the overall
> > >>> protocol, details of each RPC, etc.
> > >>>
> > >>> * A KIP describing the controller changes, how metadata is stored,
> etc.
> > >>>
> > >>> There may be other KIPs that we need (for example, if we find
> another tool
> > >>> that still has a hard ZK dependency), but that's the general idea.
> KIP-500
> > >>> is about the overall design-- the follow on KIPs are about the
> specific
> > >>> details.
> > >>>
> > >>> best,
> > >>> Colin
> > >>>
> > >>>
> > >>>>
> > >>>>> On Thu, Aug 22, 2019, 11:58 AM Colin McCabe <cm...@apache.org>
> wrote:
> > >>>>>
> > >>>>>> On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> > >>>>>> Thanks, Colin.  The changes you made to the KIP related to the
> bridge
> > >>>>>> release help make it clearer.  I still have some confusion about
> the
> > >>>>> phrase
> > >>>>>> "The rolling upgrade from the bridge release will take several
> > >>> steps."
> > >>>>>> This made me think you are talking about moving from the bridge
> > >>> release
> > >>>>> to
> > >>>>>> some other, newer, release that comes after the bridge release.
> But
> > >>> I
> > >>>>>> think what you are getting at is that the bridge release can be
> run
> > >>> with
> > >>>>> or
> > >>>>>> without Zookeeper -- when first upgrading to it Zookeeper remains
> in
> > >>> use,
> > >>>>>> but then there is a transition that can be made to engage the warp
> > >>>>> drive...
> > >>>>>> I mean the Controller Quorum.  So maybe the phrase should be "The
> > >>> rolling
> > >>>>>> upgrade through the bridge release -- starting with Zookeeper
> being
> > >>> in
> > >>>>> use
> > >>>>>> and ending with Zookeeper having been replaced by the Controller
> > >>> Quorum
> > >>>>> --
> > >>>>>> will take several steps."
> > >>>>>
> > >>>>> Hi Ron,
> > >>>>>
> > >>>>> To clarify, the bridge release will require ZooKeeper.  It will
> also
> > >>> not
> > >>>>> support the controller quorum.  It's a bridge in the sense that you
> > >>> must
> > >>>>> upgrade to a bridge release prior to upgrading to a ZK-less
> release.  I
> > >>>>> added some more descriptive text to the bridge release paragraph--
> > >>>>> hopefully this makes it clearer.
> > >>>>>
> > >>>>> best,
> > >>>>> Colin
> > >>>>>
> > >>>>>>
> > >>>>>> Do I understand it correctly, and might some change in phrasing or
> > >>>>>> additional clarification help others avoid the same confusion I
> had?
> > >>>>>>
> > >>>>>> Ron
> > >>>>>>
> > >>>>>> On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org>
> > >>> wrote:
> > >>>>>>
> > >>>>>>>> On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> > >>>>>>>> Hi Colin.  I like the concept of a "bridge release" for
> migrating
> > >>>>> off of
> > >>>>>>>> Zookeeper, but I worry that it may become a bottleneck if people
> > >>>>> hesitate
> > >>>>>>>> to replace Zookeeper -- they would be unable to adopt newer
> > >>> versions
> > >>>>> of
> > >>>>>>>> Kafka until taking (what feels to them like) a giant leap.  As
> an
> > >>>>>>> example,
> > >>>>>>>> assuming version 4.0.x of Kafka is the supported bridge release,
> > >>> I
> > >>>>> would
> > >>>>>>>> not be surprised if uptake of the 4.x release and the time-based
> > >>>>> releases
> > >>>>>>>> that follow it end up being much slower due to the perceived
> > >>> barrier.
> > >>>>>>>>
> > >>>>>>>> Any perceived barrier could be lowered if the 4.0.x release
> could
> > >>>>>>>> optionally continue to use Zookeeper -- then the cutover would
> > >>> be two
> > >>>>>>>> incremental steps (move to 4.0.x, then replace Zookeeper while
> > >>>>> staying on
> > >>>>>>>> 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and
> > >>> replace
> > >>>>>>>> Zookeeper in one fell swoop).
> > >>>>>>>
> > >>>>>>> Hi Ron,
> > >>>>>>>
> > >>>>>>> Just to clarify, the "bridge release" will continue to use
> > >>> ZooKeeper.
> > >>>>> It
> > >>>>>>> will not support running without ZooKeeper.  It is the releases
> > >>> that
> > >>>>> follow
> > >>>>>>> the bridge release that will remove ZooKeeper.
> > >>>>>>>
> > >>>>>>> Also, it's a bit unclear whether the bridge release would be 3.x
> or
> > >>>>> 4.x,
> > >>>>>>> or something to follow.  We do know that the bridge release can't
> > >>> be a
> > >>>>> 2.x
> > >>>>>>> release, since it requires at least one incompatible change,
> > >>> removing
> > >>>>>>> --zookeeper options from all the shell scripts.  (Since we're
> doing
> > >>>>>>> semantic versioning, any time we make an incompatible change, we
> > >>> bump
> > >>>>> the
> > >>>>>>> major version number.)
> > >>>>>>>
> > >>>>>>> In general, using two sources of metadata is a lot more complex
> and
> > >>>>>>> error-prone than one.  A lot of the bugs and corner cases we have
> > >>> are
> > >>>>> the
> > >>>>>>> result of divergences between the controller and the state in
> > >>>>> ZooKeeper.
> > >>>>>>> Eliminating this divergence, and the split-brain scenarios it
> > >>> creates,
> > >>>>> is a
> > >>>>>>> major goal of this work.
> > >>>>>>>
> > >>>>>>>>
> > >>>>>>>> Regardless of whether what I wrote above has merit or not, I
> > >>> think
> > >>>>> the
> > >>>>>>> KIP
> > >>>>>>>> should be more explicit about what the upgrade constraints
> > >>> actually
> > >>>>> are.
> > >>>>>>>> Can the bridge release be adopted with Zookeeper remaining in
> > >>> place
> > >>>>> and
> > >>>>>>>> then cutting over as a second, follow-on step, or must the
> > >>> Controller
> > >>>>>>>> Quorum nodes be started first and the bridge release cannot be
> > >>> used
> > >>>>> with
> > >>>>>>>> Zookeeper at all?
> > >>>>>>>
> > >>>>>>> As I mentioned above, the bridge release supports (indeed,
> > >>> requires)
> > >>>>>>> ZooKeeper.  I have added a little more text about this to KIP-500
> > >>> which
> > >>>>>>> hopefully makes it clearer.
> > >>>>>>>
> > >>>>>>> best,
> > >>>>>>> Colin
> > >>>>>>>
> > >>>>>>>> If the bridge release cannot be used with Zookeeper at
> > >>>>>>>> all, then no version at or beyond the bridge release is
> available
> > >>>>>>>> unless/until abandoning Zookeeper; if the bridge release can be
> > >>> used
> > >>>>> with
> > >>>>>>>> Zookeeper, then is it the only version that can be used with
> > >>>>> Zookeeper,
> > >>>>>>> or
> > >>>>>>>> can Zookeeper be kept for additional releases if desired?
> > >>>>>>>>
> > >>>>>>>> Ron
> > >>>>>>>>
> > >>>>>>>> On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <
> > >>> rndgstn@gmail.com>
> > >>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> Hi Colin.  The diagram up at the top confused me --
> > >>> specifically,
> > >>>>> the
> > >>>>>>>>> lines connecting the controller/active-controller to the
> > >>> brokers.
> > >>>>> I
> > >>>>>>> had
> > >>>>>>>>> assumed the arrows on those lines represented the direction of
> > >>> data
> > >>>>>>> flow,
> > >>>>>>>>> but that is not the case; the arrows actually identify the
> > >>> target
> > >>>>> of
> > >>>>>>> the
> > >>>>>>>>> action, and the non-arrowed end indicates the initiator of the
> > >>>>>>> action.  For
> > >>>>>>>>> example, the lines point from the controller to the brokers in
> > >>> the
> > >>>>>>> "today"
> > >>>>>>>>> section on the left to show that the controller pushes to the
> > >>>>> brokers;
> > >>>>>>> the
> > >>>>>>>>> lines point from the brokers to the active-controller in the
> > >>>>> "tomorrow"
> > >>>>>>>>> section on the right to show that the brokers pull from the
> > >>>>>>>>> active-controller.  As I said, this confused me because my gut
> > >>>>>>> instinct was
> > >>>>>>>>> to interpret the arrow as indicating the direction of data
> > >>> flow,
> > >>>>> and
> > >>>>>>> when I
> > >>>>>>>>> look at the "tomorrow" picture on the right I initially thought
> > >>>>>>> information
> > >>>>>>>>> was moving from the brokers to the active-controller.  Did you
> > >>>>> consider
> > >>>>>>>>> drawing that picture with the arrows reversed in the "tomorrow"
> > >>>>> side so
> > >>>>>>>>> that the arrows represent the direction of data flow, and then
> > >>> add
> > >>>>> the
> > >>>>>>>>> labels "push" on the "today" side and "pull" on the "tomorrow"
> > >>>>> side to
> > >>>>>>>>> indicate who initiates the data flow?  It occurs to me that
> > >>> this
> > >>>>>>> picture
> > >>>>>>>>> may end up being widely distributed, so it might be in
> > >>> everyone's
> > >>>>>>> interest
> > >>>>>>>>> to proactively avoid any possible confusion by being more
> > >>> explicit.
> > >>>>>>>>>
> > >>>>>>>>> Minor corrections?
> > >>>>>>>>> <<<In the current world, a broker which can contact ZooKeeper
> > >>> but
> > >>>>> which
> > >>>>>>>>> is partitioned from the active controller
> > >>>>>>>>>>>> In the current world, a broker which can contact ZooKeeper
> > >>> but
> > >>>>> which
> > >>>>>>>>> is partitioned from the controller
> > >>>>>>>>>
> > >>>>>>>>> <<<Eventually, the controller will ask the broker to finally go
> > >>>>> offline
> > >>>>>>>>>>>> Eventually, the active controller will ask the broker to
> > >>>>> finally go
> > >>>>>>>>> offline
> > >>>>>>>>>
> > >>>>>>>>> <<<New versions of the clients should send these operations
> > >>>>> directly to
> > >>>>>>>>> the controller
> > >>>>>>>>>>>> New versions of the clients should send these operations
> > >>>>> directly to
> > >>>>>>>>> the active controller
> > >>>>>>>>>
> > >>>>>>>>> <<<In the post-ZK world, the leader will make an RPC to the
> > >>>>> controller
> > >>>>>>>>> instead
> > >>>>>>>>>>>> In the post-ZK world, the leader will make an RPC to the
> > >>> active
> > >>>>>>>>> controller instead
> > >>>>>>>>>
> > >>>>>>>>> <<<For example, the brokers may need to forward their requests
> > >>> to
> > >>>>> the
> > >>>>>>>>> controller.
> > >>>>>>>>>>>> For example, the brokers may need to forward their requests
> > >>> to
> > >>>>> the
> > >>>>>>>>> active controller.
> > >>>>>>>>>
> > >>>>>>>>> <<<The new controller will monitor ZooKeeper for legacy broker
> > >>> node
> > >>>>>>>>> registrations
> > >>>>>>>>>>>> The new (active) controller will monitor ZooKeeper for
> > >>> legacy
> > >>>>> broker
> > >>>>>>>>> node registrations
> > >>>>>>>>>
> > >>>>>>>>> Ron
> > >>>>>>>>>
> > >>>>>>>>> On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <
> > >>> cmccabe@apache.org>
> > >>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Hi all,
> > >>>>>>>>>>
> > >>>>>>>>>> The KIP has been out for a while, so I'm thinking about
> > >>> calling a
> > >>>>> vote
> > >>>>>>>>>> some time this week.
> > >>>>>>>>>>
> > >>>>>>>>>> best,
> > >>>>>>>>>> Colin
> > >>>>>>>>>>
> > >>>>>>>>>>> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > >>>>>>>>>>>> On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > >>>>>>>>>>>> Thanks for the KIP, Colin. This looks great!
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> I really like the idea of separating the Controller and
> > >>> Broker
> > >>>>>>> JVMs.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> As you alluded to above, it might be nice to have a
> > >>> separate
> > >>>>>>>>>>>> broker-registration API to avoid overloading the metadata
> > >>>>> fetch
> > >>>>>>> API.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Hi David,
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks for taking a look.
> > >>>>>>>>>>>
> > >>>>>>>>>>> I removed the sentence about MetadataFetch also serving as
> > >>> the
> > >>>>>>> broker
> > >>>>>>>>>>> registration API.  I think I agree that we will probably
> > >>> want a
> > >>>>>>>>>>> separate RPC to fill this role.  We will have a follow-on
> > >>> KIP
> > >>>>> that
> > >>>>>>> will
> > >>>>>>>>>>> go into more detail about metadata propagation and
> > >>> registration
> > >>>>> in
> > >>>>>>> the
> > >>>>>>>>>>> post-ZK world.  That KIP will also have a full description
> > >>> of
> > >>>>> the
> > >>>>>>>>>>> registration RPC, etc.  For now, I think the important part
> > >>> for
> > >>>>>>> KIP-500
> > >>>>>>>>>>> is that the broker registers with the controller quorum.  On
> > >>>>>>>>>>> registration, the controller quorum assigns it a new broker
> > >>>>> epoch,
> > >>>>>>>>>>> which can distinguish successive broker incarnations.
> > >>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> When a broker gets a metadata delta, will it be a
> > >>> sequence of
> > >>>>>>> deltas
> > >>>>>>>>>> since
> > >>>>>>>>>>>> the last update or a cumulative delta since the last
> > >>> update?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> It will be a sequence of deltas.  Basically, the broker
> > >>> will be
> > >>>>>>> reading
> > >>>>>>>>>>> from the metadata log.
> > >>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Will we include any kind of integrity check on the deltas
> > >>> to
> > >>>>>>> ensure
> > >>>>>>>>>> the brokers
> > >>>>>>>>>>>> have applied them correctly? Perhaps this will be
> > >>> addressed in
> > >>>>>>> one of
> > >>>>>>>>>> the
> > >>>>>>>>>>>> follow-on KIPs.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> In general, we will have checksums on the metadata that we
> > >>>>> fetch.
> > >>>>>>> This
> > >>>>>>>>>>> is similar to how we have checksums on regular data.  Or if
> > >>> the
> > >>>>>>>>>>> question is about catching logic errors in the metadata
> > >>> handling
> > >>>>>>> code,
> > >>>>>>>>>>> that sounds more like something that should be caught by
> > >>> test
> > >>>>> cases.
> > >>>>>>>>>>>
> > >>>>>>>>>>> best,
> > >>>>>>>>>>> Colin
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Thanks!
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <
> > >>>>> cmccabe@apache.org>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> Hi Mickael,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks for taking a look.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I don't think we want to support that kind of
> > >>> multi-tenancy
> > >>>>> at
> > >>>>>>> the
> > >>>>>>>>>>>>> controller level.  If the cluster is small enough that
> > >>> we
> > >>>>> want
> > >>>>>>> to
> > >>>>>>>>>> pack the
> > >>>>>>>>>>>>> controller(s) with something else, we could run them
> > >>>>> alongside
> > >>>>>>> the
> > >>>>>>>>>> brokers,
> > >>>>>>>>>>>>> or possibly inside three of the broker JVMs.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> best,
> > >>>>>>>>>>>>> Colin
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > >>>>>>>>>>>>>> Thank Colin for kickstarting this initiative.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Just one question.
> > >>>>>>>>>>>>>> - A nice feature of Zookeeper is the ability to use
> > >>>>> chroots
> > >>>>>>> and
> > >>>>>>>>>> have
> > >>>>>>>>>>>>>> several Kafka clusters use the same Zookeeper
> > >>> ensemble. Is
> > >>>>>>> this
> > >>>>>>>>>>>>>> something we should keep?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> > >>>>>>> cmccabe@apache.org>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > >>>>>>>>>>>>>>>> Hi Colin,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks for the KIP.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Currently ZooKeeper provides a convenient
> > >>> notification
> > >>>>>>>>>> mechanism for
> > >>>>>>>>>>>>>>>> knowing that broker and topic configuration has
> > >>>>> changed.
> > >>>>>>> While
> > >>>>>>>>>>>>> KIP-500 does
> > >>>>>>>>>>>>>>>> suggest that incremental metadata update is
> > >>> expected
> > >>>>> to
> > >>>>>>> come
> > >>>>>>>>>> to
> > >>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>> eventually, that would seem to imply that for some
> > >>>>> number
> > >>>>>>> of
> > >>>>>>>>>>>>> releases there
> > >>>>>>>>>>>>>>>> would be no equivalent mechanism for knowing about
> > >>>>> config
> > >>>>>>>>>> changes.
> > >>>>>>>>>>>>> Is there
> > >>>>>>>>>>>>>>>> any thinking at this point about how a similar
> > >>>>>>> notification
> > >>>>>>>>>> might be
> > >>>>>>>>>>>>>>>> provided in the future?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> We could eventually have some inotify-like mechanism
> > >>>>> where
> > >>>>>>>>>> clients
> > >>>>>>>>>>>>> could register interest in various types of events and
> > >>> got
> > >>>>>>> notified
> > >>>>>>>>>> when
> > >>>>>>>>>>>>> they happened.  Reading the metadata log is conceptually
> > >>>>> simple.
> > >>>>>>>>>> The main
> > >>>>>>>>>>>>> complexity would be in setting up an API that made
> > >>> sense and
> > >>>>>>> that
> > >>>>>>>>>> didn't
> > >>>>>>>>>>>>> unduly constrain future implementations.  We'd have to
> > >>> think
> > >>>>>>>>>> carefully
> > >>>>>>>>>>>>> about what the real use-cases for this were, though.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> best,
> > >>>>>>>>>>>>>>> Colin
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Tom
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Mon, Aug 5, 2019 at 3:49 PM Viktor
> > >>> Somogyi-Vass <
> > >>>>>>>>>>>>> viktorsomogyi@gmail.com>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Hey Colin,
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> I think this is a long-awaited KIP, thanks for
> > >>>>> driving
> > >>>>>>> it.
> > >>>>>>>>>> I'm
> > >>>>>>>>>>>>> excited to
> > >>>>>>>>>>>>>>>>> see this in Kafka once. I collected my questions
> > >>>>> (and I
> > >>>>>>>>>> accept the
> > >>>>>>>>>>>>> "TBD"
> > >>>>>>>>>>>>>>>>> answer as they might be a bit deep for this high
> > >>>>> level
> > >>>>>>> :) ).
> > >>>>>>>>>>>>>>>>> 1.) Are there any specific reasons for the
> > >>>>> Controller
> > >>>>>>> just
> > >>>>>>>>>>>>> periodically
> > >>>>>>>>>>>>>>>>> persisting its state on disk periodically
> > >>> instead of
> > >>>>>>>>>>>>> asynchronously with
> > >>>>>>>>>>>>>>>>> every update? Wouldn't less frequent saves
> > >>> increase
> > >>>>> the
> > >>>>>>>>>> chance for
> > >>>>>>>>>>>>> missing
> > >>>>>>>>>>>>>>>>> a state change if the controller crashes
> > >>> between two
> > >>>>>>> saves?
> > >>>>>>>>>>>>>>>>> 2.) Why can't we allow brokers to fetch metadata
> > >>>>> from
> > >>>>>>> the
> > >>>>>>>>>> follower
> > >>>>>>>>>>>>>>>>> controllers? I assume that followers would have
> > >>>>>>> up-to-date
> > >>>>>>>>>>>>> information
> > >>>>>>>>>>>>>>>>> therefore brokers could fetch from there in
> > >>> theory.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>> Viktor
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > >>>>>>>>>>>>> reluctanthero104@gmail.com>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Thanks for explaining Ismael! Breaking down
> > >>> into
> > >>>>>>>>>> follow-up KIPs
> > >>>>>>>>>>>>> sounds
> > >>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>> a good idea.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > >>>>>>>>>> ismael@juma.me.uk>
> > >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Hi Boyang,
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Yes, there will be several KIPs that will
> > >>>>> discuss
> > >>>>>>> the
> > >>>>>>>>>> items you
> > >>>>>>>>>>>>>>>>> describe
> > >>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>> detail. Colin, it may be helpful to make
> > >>> this
> > >>>>> clear
> > >>>>>>> in
> > >>>>>>>>>> the KIP
> > >>>>>>>>>>>>> 500
> > >>>>>>>>>>>>>>>>>>> description.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Ismael
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > >>>>>>>>>>>>> reluctanthero104@gmail.com
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Thanks Colin for initiating this important
> > >>>>> effort!
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> One question I have is whether we have a
> > >>>>> session
> > >>>>>>>>>> discussing
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> controller
> > >>>>>>>>>>>>>>>>>>>> failover in the new architecture? I know
> > >>> we
> > >>>>> are
> > >>>>>>> using
> > >>>>>>>>>> Raft
> > >>>>>>>>>>>>> protocol
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> failover, yet it's still valuable to
> > >>> discuss
> > >>>>> the
> > >>>>>>>>>> steps new
> > >>>>>>>>>>>>> cluster is
> > >>>>>>>>>>>>>>>>>>> going
> > >>>>>>>>>>>>>>>>>>>> to take to reach the stable stage again,
> > >>> so
> > >>>>> that
> > >>>>>>> we
> > >>>>>>>>>> could
> > >>>>>>>>>>>>> easily
> > >>>>>>>>>>>>>>>>>> measure
> > >>>>>>>>>>>>>>>>>>>> the availability of the metadata servers.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Another suggestion I have is to write a
> > >>>>>>> step-by-step
> > >>>>>>>>>> design
> > >>>>>>>>>>>>> doc like
> > >>>>>>>>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>>>>>> we did in KIP-98
> > >>>>>>>>>>>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >>>>>>>>>>>>>>>>>>>>> ,
> > >>>>>>>>>>>>>>>>>>>> including the new request protocols and
> > >>> how
> > >>>>> they
> > >>>>>>> are
> > >>>>>>>>>>>>> interacting in
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>> cluster. For a complicated change like
> > >>> this,
> > >>>>> an
> > >>>>>>>>>>>>> implementation design
> > >>>>>>>>>>>>>>>>>> doc
> > >>>>>>>>>>>>>>>>>>>> help a lot in the review process,
> > >>> otherwise
> > >>>>> most
> > >>>>>>>>>> discussions
> > >>>>>>>>>>>>> we have
> > >>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>> focus on high level and lose important
> > >>>>> details as
> > >>>>>>> we
> > >>>>>>>>>>>>> discover them in
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> post-agreement phase.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Boyang
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Fri, Aug 2, 2019 at 5:17 PM Colin
> > >>> McCabe <
> > >>>>>>>>>>>>> cmccabe@apache.org>
> > >>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Fri, Aug 2, 2019, at 16:33, Jose
> > >>> Armando
> > >>>>>>> Garcia
> > >>>>>>>>>> Sancio
> > >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>> Thanks Colin for the detail KIP. I
> > >>> have a
> > >>>>> few
> > >>>>>>>>>> comments
> > >>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>> questions.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> In the KIP's Motivation and Overview
> > >>> you
> > >>>>>>>>>> mentioned the
> > >>>>>>>>>>>>>>>>> LeaderAndIsr
> > >>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>> UpdateMetadata RPC. For example,
> > >>> "updates
> > >>>>>>> which
> > >>>>>>>>>> the
> > >>>>>>>>>>>>> controller
> > >>>>>>>>>>>>>>>>>>> pushes,
> > >>>>>>>>>>>>>>>>>>>>> such
> > >>>>>>>>>>>>>>>>>>>>>> as LeaderAndIsr and UpdateMetadata
> > >>>>> messages".
> > >>>>>>> Is
> > >>>>>>>>>> your
> > >>>>>>>>>>>>> thinking
> > >>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>> use MetadataFetch as a replacement to
> > >>> just
> > >>>>>>>>>>>>> UpdateMetadata only
> > >>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>> add
> > >>>>>>>>>>>>>>>>>>>>>> topic configuration in this state?
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Hi Jose,
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Thanks for taking a look.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> The goal is for MetadataFetchRequest to
> > >>>>> replace
> > >>>>>>> both
> > >>>>>>>>>>>>>>>>>>> LeaderAndIsrRequest
> > >>>>>>>>>>>>>>>>>>>>> and UpdateMetadataRequest.  Topic
> > >>>>> configurations
> > >>>>>>>>>> would be
> > >>>>>>>>>>>>> fetched
> > >>>>>>>>>>>>>>>>>> along
> > >>>>>>>>>>>>>>>>>>>>> with the other metadata.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> In the section "Broker Metadata
> > >>>>> Management",
> > >>>>>>> you
> > >>>>>>>>>> mention
> > >>>>>>>>>>>>> "Just
> > >>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>> fetch request, the broker will track
> > >>> the
> > >>>>>>> offset
> > >>>>>>>>>> of the
> > >>>>>>>>>>>>> last
> > >>>>>>>>>>>>>>>>> updates
> > >>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>> fetched". To keep the log consistent
> > >>> Raft
> > >>>>>>>>>> requires that
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> followers
> > >>>>>>>>>>>>>>>>>>>>> keep
> > >>>>>>>>>>>>>>>>>>>>>> all of the log entries (term/epoch and
> > >>>>> offset)
> > >>>>>>>>>> that are
> > >>>>>>>>>>>>> after the
> > >>>>>>>>>>>>>>>>>>>>>> highwatermark. Any log entry before
> > >>> the
> > >>>>>>>>>> highwatermark
> > >>>>>>>>>>>>> can be
> > >>>>>>>>>>>>>>>>>>>>>> compacted/snapshot. Do we expect the
> > >>>>>>>>>> MetadataFetch API
> > >>>>>>>>>>>>> to only
> > >>>>>>>>>>>>>>>>>> return
> > >>>>>>>>>>>>>>>>>>>> log
> > >>>>>>>>>>>>>>>>>>>>>> entries up to the highwatermark?
> > >>> Unlike
> > >>>>> the
> > >>>>>>> Raft
> > >>>>>>>>>>>>> replication API
> > >>>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>> will replicate/fetch log entries
> > >>> after the
> > >>>>>>>>>> highwatermark
> > >>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>> consensus?
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Good question.  Clearly, we shouldn't
> > >>> expose
> > >>>>>>>>>> metadata
> > >>>>>>>>>>>>> updates to
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> brokers until they've been stored on a
> > >>>>> majority
> > >>>>>>> of
> > >>>>>>>>>> the
> > >>>>>>>>>>>>> Raft nodes.
> > >>>>>>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>>> most obvious way to do that, like you
> > >>>>>>> mentioned, is
> > >>>>>>>>>> to
> > >>>>>>>>>>>>> have the
> > >>>>>>>>>>>>>>>>>> brokers
> > >>>>>>>>>>>>>>>>>>>>> only fetch up to the HWM, but not
> > >>> beyond.
> > >>>>> There
> > >>>>>>>>>> might be
> > >>>>>>>>>>>>> a more
> > >>>>>>>>>>>>>>>>>> clever
> > >>>>>>>>>>>>>>>>>>>> way
> > >>>>>>>>>>>>>>>>>>>>> to do it by fetching the data, but not
> > >>>>> having
> > >>>>>>> the
> > >>>>>>>>>> brokers
> > >>>>>>>>>>>>> act on it
> > >>>>>>>>>>>>>>>>>>> until
> > >>>>>>>>>>>>>>>>>>>>> the HWM advances.  I'm not sure if
> > >>> that's
> > >>>>> worth
> > >>>>>>> it
> > >>>>>>>>>> or
> > >>>>>>>>>>>>> not.  We'll
> > >>>>>>>>>>>>>>>>>>> discuss
> > >>>>>>>>>>>>>>>>>>>>> this more in a separate KIP that just
> > >>>>> discusses
> > >>>>>>>>>> just Raft.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> In section "Broker Metadata
> > >>> Management",
> > >>>>> you
> > >>>>>>>>>> mention "the
> > >>>>>>>>>>>>>>>>>> controller
> > >>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>> send a full metadata image rather
> > >>> than a
> > >>>>>>> series of
> > >>>>>>>>>>>>> deltas". This
> > >>>>>>>>>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>>>>>>>>> doesn't go into the set of operations
> > >>> that
> > >>>>>>> need
> > >>>>>>>>>> to be
> > >>>>>>>>>>>>> supported
> > >>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>> top
> > >>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>> Raft but it would be interested if
> > >>> this
> > >>>>> "full
> > >>>>>>>>>> metadata
> > >>>>>>>>>>>>> image"
> > >>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>> express also as deltas. For example,
> > >>>>> assuming
> > >>>>>>> we
> > >>>>>>>>>> are
> > >>>>>>>>>>>>> replicating
> > >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>> map
> > >>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>> "full metadata image" could be a
> > >>> sequence
> > >>>>> of
> > >>>>>>> "put"
> > >>>>>>>>>>>>> operations
> > >>>>>>>>>>>>>>>>>> (znode
> > >>>>>>>>>>>>>>>>>>>>> create
> > >>>>>>>>>>>>>>>>>>>>>> to borrow ZK semantics).
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> The full image can definitely be
> > >>> expressed
> > >>>>> as a
> > >>>>>>> sum
> > >>>>>>>>>> of
> > >>>>>>>>>>>>> deltas.  At
> > >>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>> point, the number of deltas will get
> > >>> large
> > >>>>>>> enough
> > >>>>>>>>>> that
> > >>>>>>>>>>>>> sending a
> > >>>>>>>>>>>>>>>>> full
> > >>>>>>>>>>>>>>>>>>>> image
> > >>>>>>>>>>>>>>>>>>>>> is better, though.  One question that
> > >>> we're
> > >>>>>>> still
> > >>>>>>>>>> thinking
> > >>>>>>>>>>>>> about is
> > >>>>>>>>>>>>>>>>>> how
> > >>>>>>>>>>>>>>>>>>>>> much of this can be shared with generic
> > >>>>> Kafka
> > >>>>>>> log
> > >>>>>>>>>> code,
> > >>>>>>>>>>>>> and how
> > >>>>>>>>>>>>>>>>> much
> > >>>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>> be different.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> In section "Broker Metadata
> > >>> Management",
> > >>>>> you
> > >>>>>>>>>> mention
> > >>>>>>>>>>>>> "This
> > >>>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>> double as a heartbeat, letting the
> > >>>>> controller
> > >>>>>>>>>> know that
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> alive". In section "Broker State
> > >>>>> Machine", you
> > >>>>>>>>>> mention
> > >>>>>>>>>>>>> "The
> > >>>>>>>>>>>>>>>>>>>> MetadataFetch
> > >>>>>>>>>>>>>>>>>>>>>> API serves as this registration
> > >>>>> mechanism".
> > >>>>>>> Does
> > >>>>>>>>>> this
> > >>>>>>>>>>>>> mean that
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> MetadataFetch Request will optionally
> > >>>>> include
> > >>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>> configuration
> > >>>>>>>>>>>>>>>>>>>>>> information?
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I was originally thinking that the
> > >>>>>>>>>> MetadataFetchRequest
> > >>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>> include
> > >>>>>>>>>>>>>>>>>>>>> broker configuration information.
> > >>> Thinking
> > >>>>>>> about
> > >>>>>>>>>> this
> > >>>>>>>>>>>>> more, maybe
> > >>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>> should just have a special registration
> > >>> RPC
> > >>>>> that
> > >>>>>>>>>> contains
> > >>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>> information,
> > >>>>>>>>>>>>>>>>>>>>> to avoid sending it over the wire all
> > >>> the
> > >>>>> time.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Does this also mean that MetadataFetch
> > >>>>> request
> > >>>>>>>>>> will
> > >>>>>>>>>>>>> result in
> > >>>>>>>>>>>>>>>>>>>>>> a "write"/AppendEntries through the
> > >>> Raft
> > >>>>>>>>>> replication
> > >>>>>>>>>>>>> protocol
> > >>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>> can send the associated MetadataFetch
> > >>>>>>> Response?
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I think we should require the broker to
> > >>> be
> > >>>>> out
> > >>>>>>> of
> > >>>>>>>>>> the
> > >>>>>>>>>>>>> Offline state
> > >>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>> allowing it to fetch metadata, yes.  So
> > >>> the
> > >>>>>>> separate
> > >>>>>>>>>>>>> registration
> > >>>>>>>>>>>>>>>>> RPC
> > >>>>>>>>>>>>>>>>>>>>> should have completed first.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> In section "Broker State", you mention
> > >>>>> that a
> > >>>>>>>>>> broker can
> > >>>>>>>>>>>>>>>>> transition
> > >>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>> online after it is caught with the
> > >>>>> metadata.
> > >>>>>>> What
> > >>>>>>>>>> do you
> > >>>>>>>>>>>>> mean by
> > >>>>>>>>>>>>>>>>>>> this?
> > >>>>>>>>>>>>>>>>>>>>>> Metadata is always changing. How does
> > >>> the
> > >>>>>>> broker
> > >>>>>>>>>> know
> > >>>>>>>>>>>>> that it is
> > >>>>>>>>>>>>>>>>>>> caught
> > >>>>>>>>>>>>>>>>>>>>> up
> > >>>>>>>>>>>>>>>>>>>>>> since it doesn't participate in the
> > >>>>> consensus
> > >>>>>>> or
> > >>>>>>>>>> the
> > >>>>>>>>>>>>> advancement
> > >>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> highwatermark?
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> That's a good point.  Being "caught up"
> > >>> is
> > >>>>>>> somewhat
> > >>>>>>>>>> of a
> > >>>>>>>>>>>>> fuzzy
> > >>>>>>>>>>>>>>>>>> concept
> > >>>>>>>>>>>>>>>>>>>>> here, since the brokers do not
> > >>> participate
> > >>>>> in
> > >>>>>>> the
> > >>>>>>>>>> metadata
> > >>>>>>>>>>>>>>>>> consensus.
> > >>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>> think ideally we would want to define
> > >>> it in
> > >>>>>>> terms
> > >>>>>>>>>> of time
> > >>>>>>>>>>>>> ("the
> > >>>>>>>>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>>>> has
> > >>>>>>>>>>>>>>>>>>>>> all the updates from the last 2
> > >>> minutes",
> > >>>>> for
> > >>>>>>>>>> example.)
> > >>>>>>>>>>>>> We should
> > >>>>>>>>>>>>>>>>>>> spell
> > >>>>>>>>>>>>>>>>>>>>> this out better in the KIP.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> In section "Start the controller
> > >>> quorum
> > >>>>>>> nodes",
> > >>>>>>>>>> you
> > >>>>>>>>>>>>> mention "Once
> > >>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>> has
> > >>>>>>>>>>>>>>>>>>>>>> taken over the /controller node, the
> > >>>>> active
> > >>>>>>>>>> controller
> > >>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>> proceed
> > >>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> load
> > >>>>>>>>>>>>>>>>>>>>>> the full state of ZooKeeper.  It will
> > >>>>> write
> > >>>>>>> out
> > >>>>>>>>>> this
> > >>>>>>>>>>>>> information
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> quorum's metadata storage.  After this
> > >>>>> point,
> > >>>>>>> the
> > >>>>>>>>>>>>> metadata quorum
> > >>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>> the metadata store of record, rather
> > >>> than
> > >>>>> the
> > >>>>>>>>>> data in
> > >>>>>>>>>>>>> ZooKeeper."
> > >>>>>>>>>>>>>>>>>>>> During
> > >>>>>>>>>>>>>>>>>>>>>> this migration do should we expect to
> > >>>>> have a
> > >>>>>>>>>> small period
> > >>>>>>>>>>>>>>>>>> controller
> > >>>>>>>>>>>>>>>>>>>>>> unavailability while the controller
> > >>>>> replicas
> > >>>>>>> this
> > >>>>>>>>>> state
> > >>>>>>>>>>>>> to all of
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> raft
> > >>>>>>>>>>>>>>>>>>>>>> nodes in the controller quorum and we
> > >>>>> buffer
> > >>>>>>> new
> > >>>>>>>>>>>>> controller API
> > >>>>>>>>>>>>>>>>>>>> requests?
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Yes, the controller would be unavailable
> > >>>>> during
> > >>>>>>> this
> > >>>>>>>>>>>>> time.  I don't
> > >>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>> this will be that different from the
> > >>> current
> > >>>>>>> period
> > >>>>>>>>>> of
> > >>>>>>>>>>>>>>>>> unavailability
> > >>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>> a new controller starts up and needs to
> > >>>>> load the
> > >>>>>>>>>> full
> > >>>>>>>>>>>>> state from
> > >>>>>>>>>>>>>>>>> ZK.
> > >>>>>>>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>>> main difference is that in this period,
> > >>> we'd
> > >>>>>>> have
> > >>>>>>>>>> to write
> > >>>>>>>>>>>>> to the
> > >>>>>>>>>>>>>>>>>>>>> controller quorum rather than just to
> > >>>>> memory.
> > >>>>>>> But
> > >>>>>>>>>> we
> > >>>>>>>>>>>>> believe this
> > >>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>> be pretty fast.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> regards,
> > >>>>>>>>>>>>>>>>>>>>> Colin
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Thanks!
> > >>>>>>>>>>>>>>>>>>>>>> -Jose
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> --
> > >>>>>>>>>>>> David Arthur
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Mon, Sep 2, 2019, at 07:51, Ron Dagostino wrote:
> Hi Colin.  It is not unusual for customers to wait before upgrading — 
> to avoid so-called “point-zero” releases — to avoid as many of the 
> inevitable bugs that ride along with new functionality as possible.  
> Removal of Zookeeper is going feel especially risky to these customers, 
> and arguably it is going to feel risky even to customers who might 
> otherwise be less sensitive to upgrade risk.
> 
> This leads me to believe it is reasonable to expect that the uptake of 
> the new ZK-less consensus quorum could be delayed in many installations 
> — that such customers might wait longer than usual to adopt the feature 
> and abandon their Zookeeper servers.
> 
> Will it be possible to use releases beyond the bridge release and not 
> abandon Zookeeper?  For example, what would happen if post-bridge the 
> new consensus quorum servers are never started?  Would Kafka still work 
> fine?  At what point MUST Zookeeper be abandoned?  Taking the 
> perspective of the above customers, I think they would prefer to have 
> others adopt the new ZK-less consensus quorum for several months and 
> encounter many of the inevitable bugs before adopting it themselves.  
> But at the same time they will not want to be stuck on the bridge 
> release that whole time because there are going to be both bug fixes 
> and new features that they will want to take advantage of.
> 
> If the bridge release is the last one that supports Zookeeper, and if 
> some customers stay on that release for a while, then I could see those 
> customers wanting back-ports of bug fixes and features to occur for a 
> period of time that extends beyond what is normally done.
> 
> Basically, to sum all of the above up, I think there is a reasonable 
> probability that a single bridge release only could become a potential 
> barrier that causes angst for the project and the community.
> 
> I wonder if it would be in the interest of the project and the 
> community to mitigate the risk of there being a bridge release barrier 
> by extending the time when ZK would still be supported — perhaps for up 
> to a year — and the new co send us quorum could remain optional.
> 
> Ron
> 

Hi Ron,

Changing things always involves risk.  This is why we are trying to do as much as we can incrementally.  For example, removing ZK dependencies from tools, and from brokers.  However, there are things that can't really be done incrementally, and one of these is switching over to a new metadata store.

It might seem like supporting multiple options for where to store metadata would be safer somehow, but actually this is not the case.  Having to support totally different code paths involves a lot more work and a lot more testing.  We already have configurations that aren't tested enough.  Doubling (at least) the number of configurations we have to test is a non-starter.

This also ties in with the discussion in the KIP about why we don't plan on supporting pluggable consensus or pluggable metadata storage.  Doing this would force us to use only the least-common denominator features of every metadata storage.  We would not be able to take advantage of metadata as a stream of events, or any of the features that ZK doesn't have.  Configuration would also be quite complex.

As the KIP states, the upgrade from a bridge release (there may be several bridge releases) to ZK will have no impact on clients.  It also won't have any impact on cluster sizing (ZK nodes will simply become controller nodes).  And it will be possible to do with a rolling upgrade.  I agree that some people may be nervous about running the new software, and we may want to have more point releases of the older branches.

This is something that we'll discuss when people propose release schedules.  In general, this isn't fundamentally different than someone wanting a new release of 1.x because they don't want to upgrade to 2.x.  If there's enough interest, we'll do it.

best,
Colin

> 
> > On Aug 26, 2019, at 6:55 PM, Colin McCabe <cm...@apache.org> wrote:
> > 
> > Hi Ryanne,
> > 
> > Good point.  I added a section titled "future work" with information about the follow-on KIPs that we discussed here.
> > 
> > best,
> > Colin
> > 
> > 
> >> On Fri, Aug 23, 2019, at 13:15, Ryanne Dolan wrote:
> >> Thanks Colin, sgtm. Please make this clear in the KIP -- otherwise it is
> >> hard to nail down what we are voting for.
> >> 
> >> Ryanne
> >> 
> >> 
> >>> On Fri, Aug 23, 2019, 12:58 PM Colin McCabe <cm...@apache.org> wrote:
> >>> 
> >>>> On Fri, Aug 23, 2019, at 06:24, Ryanne Dolan wrote:
> >>>> Colin, can you outline what specifically would be in scope for this KIP
> >>> vs
> >>>> deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
> >>>> section? Is the idea to get to the bridge release with this KIP, and then
> >>>> go from there?
> >>>> 
> >>>> Ryanne
> >>>> 
> >>> 
> >>> Hi Ryanne,
> >>> 
> >>> The goal for KIP-500 is to set out an overall vision for how we will
> >>> remove ZooKeeper and transition to managing metadata via a controller
> >>> quorum.
> >>> 
> >>> We will create follow-on KIPs that will lay out the specific details of
> >>> each step.
> >>> 
> >>> * A KIP for allowing kafka-configs.sh to change topic configurations
> >>> without using ZooKeeper.  (It can already change broker configurations
> >>> without ZK)
> >>> 
> >>> * A KIP for adding APIs to replace direct ZK access by the brokers.
> >>> 
> >>> * A KIP to describe Raft replication in Kafka, including the overall
> >>> protocol, details of each RPC, etc.
> >>> 
> >>> * A KIP describing the controller changes, how metadata is stored, etc.
> >>> 
> >>> There may be other KIPs that we need (for example, if we find another tool
> >>> that still has a hard ZK dependency), but that's the general idea.  KIP-500
> >>> is about the overall design-- the follow on KIPs are about the specific
> >>> details.
> >>> 
> >>> best,
> >>> Colin
> >>> 
> >>> 
> >>>> 
> >>>>> On Thu, Aug 22, 2019, 11:58 AM Colin McCabe <cm...@apache.org> wrote:
> >>>>> 
> >>>>>> On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> >>>>>> Thanks, Colin.  The changes you made to the KIP related to the bridge
> >>>>>> release help make it clearer.  I still have some confusion about the
> >>>>> phrase
> >>>>>> "The rolling upgrade from the bridge release will take several
> >>> steps."
> >>>>>> This made me think you are talking about moving from the bridge
> >>> release
> >>>>> to
> >>>>>> some other, newer, release that comes after the bridge release.  But
> >>> I
> >>>>>> think what you are getting at is that the bridge release can be run
> >>> with
> >>>>> or
> >>>>>> without Zookeeper -- when first upgrading to it Zookeeper remains in
> >>> use,
> >>>>>> but then there is a transition that can be made to engage the warp
> >>>>> drive...
> >>>>>> I mean the Controller Quorum.  So maybe the phrase should be "The
> >>> rolling
> >>>>>> upgrade through the bridge release -- starting with Zookeeper being
> >>> in
> >>>>> use
> >>>>>> and ending with Zookeeper having been replaced by the Controller
> >>> Quorum
> >>>>> --
> >>>>>> will take several steps."
> >>>>> 
> >>>>> Hi Ron,
> >>>>> 
> >>>>> To clarify, the bridge release will require ZooKeeper.  It will also
> >>> not
> >>>>> support the controller quorum.  It's a bridge in the sense that you
> >>> must
> >>>>> upgrade to a bridge release prior to upgrading to a ZK-less release.  I
> >>>>> added some more descriptive text to the bridge release paragraph--
> >>>>> hopefully this makes it clearer.
> >>>>> 
> >>>>> best,
> >>>>> Colin
> >>>>> 
> >>>>>> 
> >>>>>> Do I understand it correctly, and might some change in phrasing or
> >>>>>> additional clarification help others avoid the same confusion I had?
> >>>>>> 
> >>>>>> Ron
> >>>>>> 
> >>>>>> On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org>
> >>> wrote:
> >>>>>> 
> >>>>>>>> On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> >>>>>>>> Hi Colin.  I like the concept of a "bridge release" for migrating
> >>>>> off of
> >>>>>>>> Zookeeper, but I worry that it may become a bottleneck if people
> >>>>> hesitate
> >>>>>>>> to replace Zookeeper -- they would be unable to adopt newer
> >>> versions
> >>>>> of
> >>>>>>>> Kafka until taking (what feels to them like) a giant leap.  As an
> >>>>>>> example,
> >>>>>>>> assuming version 4.0.x of Kafka is the supported bridge release,
> >>> I
> >>>>> would
> >>>>>>>> not be surprised if uptake of the 4.x release and the time-based
> >>>>> releases
> >>>>>>>> that follow it end up being much slower due to the perceived
> >>> barrier.
> >>>>>>>> 
> >>>>>>>> Any perceived barrier could be lowered if the 4.0.x release could
> >>>>>>>> optionally continue to use Zookeeper -- then the cutover would
> >>> be two
> >>>>>>>> incremental steps (move to 4.0.x, then replace Zookeeper while
> >>>>> staying on
> >>>>>>>> 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and
> >>> replace
> >>>>>>>> Zookeeper in one fell swoop).
> >>>>>>> 
> >>>>>>> Hi Ron,
> >>>>>>> 
> >>>>>>> Just to clarify, the "bridge release" will continue to use
> >>> ZooKeeper.
> >>>>> It
> >>>>>>> will not support running without ZooKeeper.  It is the releases
> >>> that
> >>>>> follow
> >>>>>>> the bridge release that will remove ZooKeeper.
> >>>>>>> 
> >>>>>>> Also, it's a bit unclear whether the bridge release would be 3.x or
> >>>>> 4.x,
> >>>>>>> or something to follow.  We do know that the bridge release can't
> >>> be a
> >>>>> 2.x
> >>>>>>> release, since it requires at least one incompatible change,
> >>> removing
> >>>>>>> --zookeeper options from all the shell scripts.  (Since we're doing
> >>>>>>> semantic versioning, any time we make an incompatible change, we
> >>> bump
> >>>>> the
> >>>>>>> major version number.)
> >>>>>>> 
> >>>>>>> In general, using two sources of metadata is a lot more complex and
> >>>>>>> error-prone than one.  A lot of the bugs and corner cases we have
> >>> are
> >>>>> the
> >>>>>>> result of divergences between the controller and the state in
> >>>>> ZooKeeper.
> >>>>>>> Eliminating this divergence, and the split-brain scenarios it
> >>> creates,
> >>>>> is a
> >>>>>>> major goal of this work.
> >>>>>>> 
> >>>>>>>> 
> >>>>>>>> Regardless of whether what I wrote above has merit or not, I
> >>> think
> >>>>> the
> >>>>>>> KIP
> >>>>>>>> should be more explicit about what the upgrade constraints
> >>> actually
> >>>>> are.
> >>>>>>>> Can the bridge release be adopted with Zookeeper remaining in
> >>> place
> >>>>> and
> >>>>>>>> then cutting over as a second, follow-on step, or must the
> >>> Controller
> >>>>>>>> Quorum nodes be started first and the bridge release cannot be
> >>> used
> >>>>> with
> >>>>>>>> Zookeeper at all?
> >>>>>>> 
> >>>>>>> As I mentioned above, the bridge release supports (indeed,
> >>> requires)
> >>>>>>> ZooKeeper.  I have added a little more text about this to KIP-500
> >>> which
> >>>>>>> hopefully makes it clearer.
> >>>>>>> 
> >>>>>>> best,
> >>>>>>> Colin
> >>>>>>> 
> >>>>>>>> If the bridge release cannot be used with Zookeeper at
> >>>>>>>> all, then no version at or beyond the bridge release is available
> >>>>>>>> unless/until abandoning Zookeeper; if the bridge release can be
> >>> used
> >>>>> with
> >>>>>>>> Zookeeper, then is it the only version that can be used with
> >>>>> Zookeeper,
> >>>>>>> or
> >>>>>>>> can Zookeeper be kept for additional releases if desired?
> >>>>>>>> 
> >>>>>>>> Ron
> >>>>>>>> 
> >>>>>>>> On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <
> >>> rndgstn@gmail.com>
> >>>>>>> wrote:
> >>>>>>>> 
> >>>>>>>>> Hi Colin.  The diagram up at the top confused me --
> >>> specifically,
> >>>>> the
> >>>>>>>>> lines connecting the controller/active-controller to the
> >>> brokers.
> >>>>> I
> >>>>>>> had
> >>>>>>>>> assumed the arrows on those lines represented the direction of
> >>> data
> >>>>>>> flow,
> >>>>>>>>> but that is not the case; the arrows actually identify the
> >>> target
> >>>>> of
> >>>>>>> the
> >>>>>>>>> action, and the non-arrowed end indicates the initiator of the
> >>>>>>> action.  For
> >>>>>>>>> example, the lines point from the controller to the brokers in
> >>> the
> >>>>>>> "today"
> >>>>>>>>> section on the left to show that the controller pushes to the
> >>>>> brokers;
> >>>>>>> the
> >>>>>>>>> lines point from the brokers to the active-controller in the
> >>>>> "tomorrow"
> >>>>>>>>> section on the right to show that the brokers pull from the
> >>>>>>>>> active-controller.  As I said, this confused me because my gut
> >>>>>>> instinct was
> >>>>>>>>> to interpret the arrow as indicating the direction of data
> >>> flow,
> >>>>> and
> >>>>>>> when I
> >>>>>>>>> look at the "tomorrow" picture on the right I initially thought
> >>>>>>> information
> >>>>>>>>> was moving from the brokers to the active-controller.  Did you
> >>>>> consider
> >>>>>>>>> drawing that picture with the arrows reversed in the "tomorrow"
> >>>>> side so
> >>>>>>>>> that the arrows represent the direction of data flow, and then
> >>> add
> >>>>> the
> >>>>>>>>> labels "push" on the "today" side and "pull" on the "tomorrow"
> >>>>> side to
> >>>>>>>>> indicate who initiates the data flow?  It occurs to me that
> >>> this
> >>>>>>> picture
> >>>>>>>>> may end up being widely distributed, so it might be in
> >>> everyone's
> >>>>>>> interest
> >>>>>>>>> to proactively avoid any possible confusion by being more
> >>> explicit.
> >>>>>>>>> 
> >>>>>>>>> Minor corrections?
> >>>>>>>>> <<<In the current world, a broker which can contact ZooKeeper
> >>> but
> >>>>> which
> >>>>>>>>> is partitioned from the active controller
> >>>>>>>>>>>> In the current world, a broker which can contact ZooKeeper
> >>> but
> >>>>> which
> >>>>>>>>> is partitioned from the controller
> >>>>>>>>> 
> >>>>>>>>> <<<Eventually, the controller will ask the broker to finally go
> >>>>> offline
> >>>>>>>>>>>> Eventually, the active controller will ask the broker to
> >>>>> finally go
> >>>>>>>>> offline
> >>>>>>>>> 
> >>>>>>>>> <<<New versions of the clients should send these operations
> >>>>> directly to
> >>>>>>>>> the controller
> >>>>>>>>>>>> New versions of the clients should send these operations
> >>>>> directly to
> >>>>>>>>> the active controller
> >>>>>>>>> 
> >>>>>>>>> <<<In the post-ZK world, the leader will make an RPC to the
> >>>>> controller
> >>>>>>>>> instead
> >>>>>>>>>>>> In the post-ZK world, the leader will make an RPC to the
> >>> active
> >>>>>>>>> controller instead
> >>>>>>>>> 
> >>>>>>>>> <<<For example, the brokers may need to forward their requests
> >>> to
> >>>>> the
> >>>>>>>>> controller.
> >>>>>>>>>>>> For example, the brokers may need to forward their requests
> >>> to
> >>>>> the
> >>>>>>>>> active controller.
> >>>>>>>>> 
> >>>>>>>>> <<<The new controller will monitor ZooKeeper for legacy broker
> >>> node
> >>>>>>>>> registrations
> >>>>>>>>>>>> The new (active) controller will monitor ZooKeeper for
> >>> legacy
> >>>>> broker
> >>>>>>>>> node registrations
> >>>>>>>>> 
> >>>>>>>>> Ron
> >>>>>>>>> 
> >>>>>>>>> On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <
> >>> cmccabe@apache.org>
> >>>>>>> wrote:
> >>>>>>>>> 
> >>>>>>>>>> Hi all,
> >>>>>>>>>> 
> >>>>>>>>>> The KIP has been out for a while, so I'm thinking about
> >>> calling a
> >>>>> vote
> >>>>>>>>>> some time this week.
> >>>>>>>>>> 
> >>>>>>>>>> best,
> >>>>>>>>>> Colin
> >>>>>>>>>> 
> >>>>>>>>>>> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> >>>>>>>>>>>> On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> >>>>>>>>>>>> Thanks for the KIP, Colin. This looks great!
> >>>>>>>>>>>> 
> >>>>>>>>>>>> I really like the idea of separating the Controller and
> >>> Broker
> >>>>>>> JVMs.
> >>>>>>>>>>>> 
> >>>>>>>>>>>> As you alluded to above, it might be nice to have a
> >>> separate
> >>>>>>>>>>>> broker-registration API to avoid overloading the metadata
> >>>>> fetch
> >>>>>>> API.
> >>>>>>>>>>>> 
> >>>>>>>>>>> 
> >>>>>>>>>>> Hi David,
> >>>>>>>>>>> 
> >>>>>>>>>>> Thanks for taking a look.
> >>>>>>>>>>> 
> >>>>>>>>>>> I removed the sentence about MetadataFetch also serving as
> >>> the
> >>>>>>> broker
> >>>>>>>>>>> registration API.  I think I agree that we will probably
> >>> want a
> >>>>>>>>>>> separate RPC to fill this role.  We will have a follow-on
> >>> KIP
> >>>>> that
> >>>>>>> will
> >>>>>>>>>>> go into more detail about metadata propagation and
> >>> registration
> >>>>> in
> >>>>>>> the
> >>>>>>>>>>> post-ZK world.  That KIP will also have a full description
> >>> of
> >>>>> the
> >>>>>>>>>>> registration RPC, etc.  For now, I think the important part
> >>> for
> >>>>>>> KIP-500
> >>>>>>>>>>> is that the broker registers with the controller quorum.  On
> >>>>>>>>>>> registration, the controller quorum assigns it a new broker
> >>>>> epoch,
> >>>>>>>>>>> which can distinguish successive broker incarnations.
> >>>>>>>>>>> 
> >>>>>>>>>>>> 
> >>>>>>>>>>>> When a broker gets a metadata delta, will it be a
> >>> sequence of
> >>>>>>> deltas
> >>>>>>>>>> since
> >>>>>>>>>>>> the last update or a cumulative delta since the last
> >>> update?
> >>>>>>>>>>>> 
> >>>>>>>>>>> 
> >>>>>>>>>>> It will be a sequence of deltas.  Basically, the broker
> >>> will be
> >>>>>>> reading
> >>>>>>>>>>> from the metadata log.
> >>>>>>>>>>> 
> >>>>>>>>>>>> 
> >>>>>>>>>>>> Will we include any kind of integrity check on the deltas
> >>> to
> >>>>>>> ensure
> >>>>>>>>>> the brokers
> >>>>>>>>>>>> have applied them correctly? Perhaps this will be
> >>> addressed in
> >>>>>>> one of
> >>>>>>>>>> the
> >>>>>>>>>>>> follow-on KIPs.
> >>>>>>>>>>>> 
> >>>>>>>>>>> 
> >>>>>>>>>>> In general, we will have checksums on the metadata that we
> >>>>> fetch.
> >>>>>>> This
> >>>>>>>>>>> is similar to how we have checksums on regular data.  Or if
> >>> the
> >>>>>>>>>>> question is about catching logic errors in the metadata
> >>> handling
> >>>>>>> code,
> >>>>>>>>>>> that sounds more like something that should be caught by
> >>> test
> >>>>> cases.
> >>>>>>>>>>> 
> >>>>>>>>>>> best,
> >>>>>>>>>>> Colin
> >>>>>>>>>>> 
> >>>>>>>>>>> 
> >>>>>>>>>>>> Thanks!
> >>>>>>>>>>>> 
> >>>>>>>>>>>> On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <
> >>>>> cmccabe@apache.org>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>> 
> >>>>>>>>>>>>> Hi Mickael,
> >>>>>>>>>>>>> 
> >>>>>>>>>>>>> Thanks for taking a look.
> >>>>>>>>>>>>> 
> >>>>>>>>>>>>> I don't think we want to support that kind of
> >>> multi-tenancy
> >>>>> at
> >>>>>>> the
> >>>>>>>>>>>>> controller level.  If the cluster is small enough that
> >>> we
> >>>>> want
> >>>>>>> to
> >>>>>>>>>> pack the
> >>>>>>>>>>>>> controller(s) with something else, we could run them
> >>>>> alongside
> >>>>>>> the
> >>>>>>>>>> brokers,
> >>>>>>>>>>>>> or possibly inside three of the broker JVMs.
> >>>>>>>>>>>>> 
> >>>>>>>>>>>>> best,
> >>>>>>>>>>>>> Colin
> >>>>>>>>>>>>> 
> >>>>>>>>>>>>> 
> >>>>>>>>>>>>>> On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> >>>>>>>>>>>>>> Thank Colin for kickstarting this initiative.
> >>>>>>>>>>>>>> 
> >>>>>>>>>>>>>> Just one question.
> >>>>>>>>>>>>>> - A nice feature of Zookeeper is the ability to use
> >>>>> chroots
> >>>>>>> and
> >>>>>>>>>> have
> >>>>>>>>>>>>>> several Kafka clusters use the same Zookeeper
> >>> ensemble. Is
> >>>>>>> this
> >>>>>>>>>>>>>> something we should keep?
> >>>>>>>>>>>>>> 
> >>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>> 
> >>>>>>>>>>>>>> On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> >>>>>>> cmccabe@apache.org>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> >>>>>>>>>>>>>>>> Hi Colin,
> >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> Currently ZooKeeper provides a convenient
> >>> notification
> >>>>>>>>>> mechanism for
> >>>>>>>>>>>>>>>> knowing that broker and topic configuration has
> >>>>> changed.
> >>>>>>> While
> >>>>>>>>>>>>> KIP-500 does
> >>>>>>>>>>>>>>>> suggest that incremental metadata update is
> >>> expected
> >>>>> to
> >>>>>>> come
> >>>>>>>>>> to
> >>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>> eventually, that would seem to imply that for some
> >>>>> number
> >>>>>>> of
> >>>>>>>>>>>>> releases there
> >>>>>>>>>>>>>>>> would be no equivalent mechanism for knowing about
> >>>>> config
> >>>>>>>>>> changes.
> >>>>>>>>>>>>> Is there
> >>>>>>>>>>>>>>>> any thinking at this point about how a similar
> >>>>>>> notification
> >>>>>>>>>> might be
> >>>>>>>>>>>>>>>> provided in the future?
> >>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>> We could eventually have some inotify-like mechanism
> >>>>> where
> >>>>>>>>>> clients
> >>>>>>>>>>>>> could register interest in various types of events and
> >>> got
> >>>>>>> notified
> >>>>>>>>>> when
> >>>>>>>>>>>>> they happened.  Reading the metadata log is conceptually
> >>>>> simple.
> >>>>>>>>>> The main
> >>>>>>>>>>>>> complexity would be in setting up an API that made
> >>> sense and
> >>>>>>> that
> >>>>>>>>>> didn't
> >>>>>>>>>>>>> unduly constrain future implementations.  We'd have to
> >>> think
> >>>>>>>>>> carefully
> >>>>>>>>>>>>> about what the real use-cases for this were, though.
> >>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>> best,
> >>>>>>>>>>>>>>> Colin
> >>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> Tom
> >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> On Mon, Aug 5, 2019 at 3:49 PM Viktor
> >>> Somogyi-Vass <
> >>>>>>>>>>>>> viktorsomogyi@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>> Hey Colin,
> >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>> I think this is a long-awaited KIP, thanks for
> >>>>> driving
> >>>>>>> it.
> >>>>>>>>>> I'm
> >>>>>>>>>>>>> excited to
> >>>>>>>>>>>>>>>>> see this in Kafka once. I collected my questions
> >>>>> (and I
> >>>>>>>>>> accept the
> >>>>>>>>>>>>> "TBD"
> >>>>>>>>>>>>>>>>> answer as they might be a bit deep for this high
> >>>>> level
> >>>>>>> :) ).
> >>>>>>>>>>>>>>>>> 1.) Are there any specific reasons for the
> >>>>> Controller
> >>>>>>> just
> >>>>>>>>>>>>> periodically
> >>>>>>>>>>>>>>>>> persisting its state on disk periodically
> >>> instead of
> >>>>>>>>>>>>> asynchronously with
> >>>>>>>>>>>>>>>>> every update? Wouldn't less frequent saves
> >>> increase
> >>>>> the
> >>>>>>>>>> chance for
> >>>>>>>>>>>>> missing
> >>>>>>>>>>>>>>>>> a state change if the controller crashes
> >>> between two
> >>>>>>> saves?
> >>>>>>>>>>>>>>>>> 2.) Why can't we allow brokers to fetch metadata
> >>>>> from
> >>>>>>> the
> >>>>>>>>>> follower
> >>>>>>>>>>>>>>>>> controllers? I assume that followers would have
> >>>>>>> up-to-date
> >>>>>>>>>>>>> information
> >>>>>>>>>>>>>>>>> therefore brokers could fetch from there in
> >>> theory.
> >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>> Viktor
> >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>> On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> >>>>>>>>>>>>> reluctanthero104@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>> Thanks for explaining Ismael! Breaking down
> >>> into
> >>>>>>>>>> follow-up KIPs
> >>>>>>>>>>>>> sounds
> >>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>> a good idea.
> >>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>> On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> >>>>>>>>>> ismael@juma.me.uk>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>> Hi Boyang,
> >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>> Yes, there will be several KIPs that will
> >>>>> discuss
> >>>>>>> the
> >>>>>>>>>> items you
> >>>>>>>>>>>>>>>>> describe
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> detail. Colin, it may be helpful to make
> >>> this
> >>>>> clear
> >>>>>>> in
> >>>>>>>>>> the KIP
> >>>>>>>>>>>>> 500
> >>>>>>>>>>>>>>>>>>> description.
> >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>> Ismael
> >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>> On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> >>>>>>>>>>>>> reluctanthero104@gmail.com
> >>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>> Thanks Colin for initiating this important
> >>>>> effort!
> >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>> One question I have is whether we have a
> >>>>> session
> >>>>>>>>>> discussing
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> controller
> >>>>>>>>>>>>>>>>>>>> failover in the new architecture? I know
> >>> we
> >>>>> are
> >>>>>>> using
> >>>>>>>>>> Raft
> >>>>>>>>>>>>> protocol
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> failover, yet it's still valuable to
> >>> discuss
> >>>>> the
> >>>>>>>>>> steps new
> >>>>>>>>>>>>> cluster is
> >>>>>>>>>>>>>>>>>>> going
> >>>>>>>>>>>>>>>>>>>> to take to reach the stable stage again,
> >>> so
> >>>>> that
> >>>>>>> we
> >>>>>>>>>> could
> >>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>>> measure
> >>>>>>>>>>>>>>>>>>>> the availability of the metadata servers.
> >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>> Another suggestion I have is to write a
> >>>>>>> step-by-step
> >>>>>>>>>> design
> >>>>>>>>>>>>> doc like
> >>>>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>> we did in KIP-98
> >>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>> 
> >>>>>>>>>> 
> >>>>>>> 
> >>>>> 
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >>>>>>>>>>>>>>>>>>>>> ,
> >>>>>>>>>>>>>>>>>>>> including the new request protocols and
> >>> how
> >>>>> they
> >>>>>>> are
> >>>>>>>>>>>>> interacting in
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>> cluster. For a complicated change like
> >>> this,
> >>>>> an
> >>>>>>>>>>>>> implementation design
> >>>>>>>>>>>>>>>>>> doc
> >>>>>>>>>>>>>>>>>>>> help a lot in the review process,
> >>> otherwise
> >>>>> most
> >>>>>>>>>> discussions
> >>>>>>>>>>>>> we have
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>> focus on high level and lose important
> >>>>> details as
> >>>>>>> we
> >>>>>>>>>>>>> discover them in
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> post-agreement phase.
> >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>> Boyang
> >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>> On Fri, Aug 2, 2019 at 5:17 PM Colin
> >>> McCabe <
> >>>>>>>>>>>>> cmccabe@apache.org>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> On Fri, Aug 2, 2019, at 16:33, Jose
> >>> Armando
> >>>>>>> Garcia
> >>>>>>>>>> Sancio
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>> Thanks Colin for the detail KIP. I
> >>> have a
> >>>>> few
> >>>>>>>>>> comments
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> questions.
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> In the KIP's Motivation and Overview
> >>> you
> >>>>>>>>>> mentioned the
> >>>>>>>>>>>>>>>>> LeaderAndIsr
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> UpdateMetadata RPC. For example,
> >>> "updates
> >>>>>>> which
> >>>>>>>>>> the
> >>>>>>>>>>>>> controller
> >>>>>>>>>>>>>>>>>>> pushes,
> >>>>>>>>>>>>>>>>>>>>> such
> >>>>>>>>>>>>>>>>>>>>>> as LeaderAndIsr and UpdateMetadata
> >>>>> messages".
> >>>>>>> Is
> >>>>>>>>>> your
> >>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>> use MetadataFetch as a replacement to
> >>> just
> >>>>>>>>>>>>> UpdateMetadata only
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>>> topic configuration in this state?
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> Hi Jose,
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> Thanks for taking a look.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> The goal is for MetadataFetchRequest to
> >>>>> replace
> >>>>>>> both
> >>>>>>>>>>>>>>>>>>> LeaderAndIsrRequest
> >>>>>>>>>>>>>>>>>>>>> and UpdateMetadataRequest.  Topic
> >>>>> configurations
> >>>>>>>>>> would be
> >>>>>>>>>>>>> fetched
> >>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>> with the other metadata.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> In the section "Broker Metadata
> >>>>> Management",
> >>>>>>> you
> >>>>>>>>>> mention
> >>>>>>>>>>>>> "Just
> >>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>> fetch request, the broker will track
> >>> the
> >>>>>>> offset
> >>>>>>>>>> of the
> >>>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>> updates
> >>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>> fetched". To keep the log consistent
> >>> Raft
> >>>>>>>>>> requires that
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> followers
> >>>>>>>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>>>>>>>> all of the log entries (term/epoch and
> >>>>> offset)
> >>>>>>>>>> that are
> >>>>>>>>>>>>> after the
> >>>>>>>>>>>>>>>>>>>>>> highwatermark. Any log entry before
> >>> the
> >>>>>>>>>> highwatermark
> >>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>> compacted/snapshot. Do we expect the
> >>>>>>>>>> MetadataFetch API
> >>>>>>>>>>>>> to only
> >>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>> entries up to the highwatermark?
> >>> Unlike
> >>>>> the
> >>>>>>> Raft
> >>>>>>>>>>>>> replication API
> >>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>> will replicate/fetch log entries
> >>> after the
> >>>>>>>>>> highwatermark
> >>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> consensus?
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> Good question.  Clearly, we shouldn't
> >>> expose
> >>>>>>>>>> metadata
> >>>>>>>>>>>>> updates to
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> brokers until they've been stored on a
> >>>>> majority
> >>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>>> Raft nodes.
> >>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>> most obvious way to do that, like you
> >>>>>>> mentioned, is
> >>>>>>>>>> to
> >>>>>>>>>>>>> have the
> >>>>>>>>>>>>>>>>>> brokers
> >>>>>>>>>>>>>>>>>>>>> only fetch up to the HWM, but not
> >>> beyond.
> >>>>> There
> >>>>>>>>>> might be
> >>>>>>>>>>>>> a more
> >>>>>>>>>>>>>>>>>> clever
> >>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>> to do it by fetching the data, but not
> >>>>> having
> >>>>>>> the
> >>>>>>>>>> brokers
> >>>>>>>>>>>>> act on it
> >>>>>>>>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>> the HWM advances.  I'm not sure if
> >>> that's
> >>>>> worth
> >>>>>>> it
> >>>>>>>>>> or
> >>>>>>>>>>>>> not.  We'll
> >>>>>>>>>>>>>>>>>>> discuss
> >>>>>>>>>>>>>>>>>>>>> this more in a separate KIP that just
> >>>>> discusses
> >>>>>>>>>> just Raft.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> In section "Broker Metadata
> >>> Management",
> >>>>> you
> >>>>>>>>>> mention "the
> >>>>>>>>>>>>>>>>>> controller
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>> send a full metadata image rather
> >>> than a
> >>>>>>> series of
> >>>>>>>>>>>>> deltas". This
> >>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>> doesn't go into the set of operations
> >>> that
> >>>>>>> need
> >>>>>>>>>> to be
> >>>>>>>>>>>>> supported
> >>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>> top
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> Raft but it would be interested if
> >>> this
> >>>>> "full
> >>>>>>>>>> metadata
> >>>>>>>>>>>>> image"
> >>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>> express also as deltas. For example,
> >>>>> assuming
> >>>>>>> we
> >>>>>>>>>> are
> >>>>>>>>>>>>> replicating
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>> "full metadata image" could be a
> >>> sequence
> >>>>> of
> >>>>>>> "put"
> >>>>>>>>>>>>> operations
> >>>>>>>>>>>>>>>>>> (znode
> >>>>>>>>>>>>>>>>>>>>> create
> >>>>>>>>>>>>>>>>>>>>>> to borrow ZK semantics).
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> The full image can definitely be
> >>> expressed
> >>>>> as a
> >>>>>>> sum
> >>>>>>>>>> of
> >>>>>>>>>>>>> deltas.  At
> >>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>> point, the number of deltas will get
> >>> large
> >>>>>>> enough
> >>>>>>>>>> that
> >>>>>>>>>>>>> sending a
> >>>>>>>>>>>>>>>>> full
> >>>>>>>>>>>>>>>>>>>> image
> >>>>>>>>>>>>>>>>>>>>> is better, though.  One question that
> >>> we're
> >>>>>>> still
> >>>>>>>>>> thinking
> >>>>>>>>>>>>> about is
> >>>>>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>> much of this can be shared with generic
> >>>>> Kafka
> >>>>>>> log
> >>>>>>>>>> code,
> >>>>>>>>>>>>> and how
> >>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>> be different.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> In section "Broker Metadata
> >>> Management",
> >>>>> you
> >>>>>>>>>> mention
> >>>>>>>>>>>>> "This
> >>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>> double as a heartbeat, letting the
> >>>>> controller
> >>>>>>>>>> know that
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> alive". In section "Broker State
> >>>>> Machine", you
> >>>>>>>>>> mention
> >>>>>>>>>>>>> "The
> >>>>>>>>>>>>>>>>>>>> MetadataFetch
> >>>>>>>>>>>>>>>>>>>>>> API serves as this registration
> >>>>> mechanism".
> >>>>>>> Does
> >>>>>>>>>> this
> >>>>>>>>>>>>> mean that
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> MetadataFetch Request will optionally
> >>>>> include
> >>>>>>>>>> broker
> >>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>>>>> information?
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> I was originally thinking that the
> >>>>>>>>>> MetadataFetchRequest
> >>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>>>> broker configuration information.
> >>> Thinking
> >>>>>>> about
> >>>>>>>>>> this
> >>>>>>>>>>>>> more, maybe
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> should just have a special registration
> >>> RPC
> >>>>> that
> >>>>>>>>>> contains
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> information,
> >>>>>>>>>>>>>>>>>>>>> to avoid sending it over the wire all
> >>> the
> >>>>> time.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> Does this also mean that MetadataFetch
> >>>>> request
> >>>>>>>>>> will
> >>>>>>>>>>>>> result in
> >>>>>>>>>>>>>>>>>>>>>> a "write"/AppendEntries through the
> >>> Raft
> >>>>>>>>>> replication
> >>>>>>>>>>>>> protocol
> >>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>> can send the associated MetadataFetch
> >>>>>>> Response?
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> I think we should require the broker to
> >>> be
> >>>>> out
> >>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>>> Offline state
> >>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>> allowing it to fetch metadata, yes.  So
> >>> the
> >>>>>>> separate
> >>>>>>>>>>>>> registration
> >>>>>>>>>>>>>>>>> RPC
> >>>>>>>>>>>>>>>>>>>>> should have completed first.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> In section "Broker State", you mention
> >>>>> that a
> >>>>>>>>>> broker can
> >>>>>>>>>>>>>>>>> transition
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> online after it is caught with the
> >>>>> metadata.
> >>>>>>> What
> >>>>>>>>>> do you
> >>>>>>>>>>>>> mean by
> >>>>>>>>>>>>>>>>>>> this?
> >>>>>>>>>>>>>>>>>>>>>> Metadata is always changing. How does
> >>> the
> >>>>>>> broker
> >>>>>>>>>> know
> >>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>>>>> caught
> >>>>>>>>>>>>>>>>>>>>> up
> >>>>>>>>>>>>>>>>>>>>>> since it doesn't participate in the
> >>>>> consensus
> >>>>>>> or
> >>>>>>>>>> the
> >>>>>>>>>>>>> advancement
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> highwatermark?
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> That's a good point.  Being "caught up"
> >>> is
> >>>>>>> somewhat
> >>>>>>>>>> of a
> >>>>>>>>>>>>> fuzzy
> >>>>>>>>>>>>>>>>>> concept
> >>>>>>>>>>>>>>>>>>>>> here, since the brokers do not
> >>> participate
> >>>>> in
> >>>>>>> the
> >>>>>>>>>> metadata
> >>>>>>>>>>>>>>>>> consensus.
> >>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>> think ideally we would want to define
> >>> it in
> >>>>>>> terms
> >>>>>>>>>> of time
> >>>>>>>>>>>>> ("the
> >>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>> all the updates from the last 2
> >>> minutes",
> >>>>> for
> >>>>>>>>>> example.)
> >>>>>>>>>>>>> We should
> >>>>>>>>>>>>>>>>>>> spell
> >>>>>>>>>>>>>>>>>>>>> this out better in the KIP.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> In section "Start the controller
> >>> quorum
> >>>>>>> nodes",
> >>>>>>>>>> you
> >>>>>>>>>>>>> mention "Once
> >>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>> taken over the /controller node, the
> >>>>> active
> >>>>>>>>>> controller
> >>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>> proceed
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> load
> >>>>>>>>>>>>>>>>>>>>>> the full state of ZooKeeper.  It will
> >>>>> write
> >>>>>>> out
> >>>>>>>>>> this
> >>>>>>>>>>>>> information
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> quorum's metadata storage.  After this
> >>>>> point,
> >>>>>>> the
> >>>>>>>>>>>>> metadata quorum
> >>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>> the metadata store of record, rather
> >>> than
> >>>>> the
> >>>>>>>>>> data in
> >>>>>>>>>>>>> ZooKeeper."
> >>>>>>>>>>>>>>>>>>>> During
> >>>>>>>>>>>>>>>>>>>>>> this migration do should we expect to
> >>>>> have a
> >>>>>>>>>> small period
> >>>>>>>>>>>>>>>>>> controller
> >>>>>>>>>>>>>>>>>>>>>> unavailability while the controller
> >>>>> replicas
> >>>>>>> this
> >>>>>>>>>> state
> >>>>>>>>>>>>> to all of
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> raft
> >>>>>>>>>>>>>>>>>>>>>> nodes in the controller quorum and we
> >>>>> buffer
> >>>>>>> new
> >>>>>>>>>>>>> controller API
> >>>>>>>>>>>>>>>>>>>> requests?
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> Yes, the controller would be unavailable
> >>>>> during
> >>>>>>> this
> >>>>>>>>>>>>> time.  I don't
> >>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>> this will be that different from the
> >>> current
> >>>>>>> period
> >>>>>>>>>> of
> >>>>>>>>>>>>>>>>> unavailability
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>> a new controller starts up and needs to
> >>>>> load the
> >>>>>>>>>> full
> >>>>>>>>>>>>> state from
> >>>>>>>>>>>>>>>>> ZK.
> >>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>> main difference is that in this period,
> >>> we'd
> >>>>>>> have
> >>>>>>>>>> to write
> >>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>> controller quorum rather than just to
> >>>>> memory.
> >>>>>>> But
> >>>>>>>>>> we
> >>>>>>>>>>>>> believe this
> >>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>> be pretty fast.
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> regards,
> >>>>>>>>>>>>>>>>>>>>> Colin
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>> -Jose
> >>>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>> 
> >>>>>>>>>>>>> 
> >>>>>>>>>>>> 
> >>>>>>>>>>>> 
> >>>>>>>>>>>> --
> >>>>>>>>>>>> David Arthur
> >>>>>>>>>>>> 
> >>>>>>>>>>> 
> >>>>>>>>>> 
> >>>>>>>>> 
> >>>>>>>> 
> >>>>>>> 
> >>>>>> 
> >>>>> 
> >>>> 
> >>> 
> >> 
> 
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ron Dagostino <rn...@gmail.com>.
Hi Colin.  It is not unusual for customers to wait before upgrading — to avoid so-called “point-zero” releases — to avoid as many of the inevitable bugs that ride along with new functionality as possible.  Removal of Zookeeper is going feel especially risky to these customers, and arguably it is going to feel risky even to customers who might otherwise be less sensitive to upgrade risk.

This leads me to believe it is reasonable to expect that the uptake of the new ZK-less consensus quorum could be delayed in many installations — that such customers might wait longer than usual to adopt the feature and abandon their Zookeeper servers.

Will it be possible to use releases beyond the bridge release and not abandon Zookeeper?  For example, what would happen if post-bridge the new consensus quorum servers are never started?  Would Kafka still work fine?  At what point MUST Zookeeper be abandoned?  Taking the perspective of the above customers, I think they would prefer to have others adopt the new ZK-less consensus quorum for several months and encounter many of the inevitable bugs before adopting it themselves.  But at the same time they will not want to be stuck on the bridge release that whole time because there are going to be both bug fixes and new features that they will want to take advantage of.

If the bridge release is the last one that supports Zookeeper, and if some customers stay on that release for a while, then I could see those customers wanting back-ports of bug fixes and features to occur for a period of time that extends beyond what is normally done.

Basically, to sum all of the above up, I think there is a reasonable probability that a single bridge release only could become a potential barrier that causes angst for the project and the community.

I wonder if it would be in the interest of the project and the community to mitigate the risk of there being a bridge release barrier by extending the time when ZK would still be supported — perhaps for up to a year — and the new co send us quorum could remain optional.

Ron


> On Aug 26, 2019, at 6:55 PM, Colin McCabe <cm...@apache.org> wrote:
> 
> Hi Ryanne,
> 
> Good point.  I added a section titled "future work" with information about the follow-on KIPs that we discussed here.
> 
> best,
> Colin
> 
> 
>> On Fri, Aug 23, 2019, at 13:15, Ryanne Dolan wrote:
>> Thanks Colin, sgtm. Please make this clear in the KIP -- otherwise it is
>> hard to nail down what we are voting for.
>> 
>> Ryanne
>> 
>> 
>>> On Fri, Aug 23, 2019, 12:58 PM Colin McCabe <cm...@apache.org> wrote:
>>> 
>>>> On Fri, Aug 23, 2019, at 06:24, Ryanne Dolan wrote:
>>>> Colin, can you outline what specifically would be in scope for this KIP
>>> vs
>>>> deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
>>>> section? Is the idea to get to the bridge release with this KIP, and then
>>>> go from there?
>>>> 
>>>> Ryanne
>>>> 
>>> 
>>> Hi Ryanne,
>>> 
>>> The goal for KIP-500 is to set out an overall vision for how we will
>>> remove ZooKeeper and transition to managing metadata via a controller
>>> quorum.
>>> 
>>> We will create follow-on KIPs that will lay out the specific details of
>>> each step.
>>> 
>>> * A KIP for allowing kafka-configs.sh to change topic configurations
>>> without using ZooKeeper.  (It can already change broker configurations
>>> without ZK)
>>> 
>>> * A KIP for adding APIs to replace direct ZK access by the brokers.
>>> 
>>> * A KIP to describe Raft replication in Kafka, including the overall
>>> protocol, details of each RPC, etc.
>>> 
>>> * A KIP describing the controller changes, how metadata is stored, etc.
>>> 
>>> There may be other KIPs that we need (for example, if we find another tool
>>> that still has a hard ZK dependency), but that's the general idea.  KIP-500
>>> is about the overall design-- the follow on KIPs are about the specific
>>> details.
>>> 
>>> best,
>>> Colin
>>> 
>>> 
>>>> 
>>>>> On Thu, Aug 22, 2019, 11:58 AM Colin McCabe <cm...@apache.org> wrote:
>>>>> 
>>>>>> On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
>>>>>> Thanks, Colin.  The changes you made to the KIP related to the bridge
>>>>>> release help make it clearer.  I still have some confusion about the
>>>>> phrase
>>>>>> "The rolling upgrade from the bridge release will take several
>>> steps."
>>>>>> This made me think you are talking about moving from the bridge
>>> release
>>>>> to
>>>>>> some other, newer, release that comes after the bridge release.  But
>>> I
>>>>>> think what you are getting at is that the bridge release can be run
>>> with
>>>>> or
>>>>>> without Zookeeper -- when first upgrading to it Zookeeper remains in
>>> use,
>>>>>> but then there is a transition that can be made to engage the warp
>>>>> drive...
>>>>>> I mean the Controller Quorum.  So maybe the phrase should be "The
>>> rolling
>>>>>> upgrade through the bridge release -- starting with Zookeeper being
>>> in
>>>>> use
>>>>>> and ending with Zookeeper having been replaced by the Controller
>>> Quorum
>>>>> --
>>>>>> will take several steps."
>>>>> 
>>>>> Hi Ron,
>>>>> 
>>>>> To clarify, the bridge release will require ZooKeeper.  It will also
>>> not
>>>>> support the controller quorum.  It's a bridge in the sense that you
>>> must
>>>>> upgrade to a bridge release prior to upgrading to a ZK-less release.  I
>>>>> added some more descriptive text to the bridge release paragraph--
>>>>> hopefully this makes it clearer.
>>>>> 
>>>>> best,
>>>>> Colin
>>>>> 
>>>>>> 
>>>>>> Do I understand it correctly, and might some change in phrasing or
>>>>>> additional clarification help others avoid the same confusion I had?
>>>>>> 
>>>>>> Ron
>>>>>> 
>>>>>> On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org>
>>> wrote:
>>>>>> 
>>>>>>>> On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
>>>>>>>> Hi Colin.  I like the concept of a "bridge release" for migrating
>>>>> off of
>>>>>>>> Zookeeper, but I worry that it may become a bottleneck if people
>>>>> hesitate
>>>>>>>> to replace Zookeeper -- they would be unable to adopt newer
>>> versions
>>>>> of
>>>>>>>> Kafka until taking (what feels to them like) a giant leap.  As an
>>>>>>> example,
>>>>>>>> assuming version 4.0.x of Kafka is the supported bridge release,
>>> I
>>>>> would
>>>>>>>> not be surprised if uptake of the 4.x release and the time-based
>>>>> releases
>>>>>>>> that follow it end up being much slower due to the perceived
>>> barrier.
>>>>>>>> 
>>>>>>>> Any perceived barrier could be lowered if the 4.0.x release could
>>>>>>>> optionally continue to use Zookeeper -- then the cutover would
>>> be two
>>>>>>>> incremental steps (move to 4.0.x, then replace Zookeeper while
>>>>> staying on
>>>>>>>> 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and
>>> replace
>>>>>>>> Zookeeper in one fell swoop).
>>>>>>> 
>>>>>>> Hi Ron,
>>>>>>> 
>>>>>>> Just to clarify, the "bridge release" will continue to use
>>> ZooKeeper.
>>>>> It
>>>>>>> will not support running without ZooKeeper.  It is the releases
>>> that
>>>>> follow
>>>>>>> the bridge release that will remove ZooKeeper.
>>>>>>> 
>>>>>>> Also, it's a bit unclear whether the bridge release would be 3.x or
>>>>> 4.x,
>>>>>>> or something to follow.  We do know that the bridge release can't
>>> be a
>>>>> 2.x
>>>>>>> release, since it requires at least one incompatible change,
>>> removing
>>>>>>> --zookeeper options from all the shell scripts.  (Since we're doing
>>>>>>> semantic versioning, any time we make an incompatible change, we
>>> bump
>>>>> the
>>>>>>> major version number.)
>>>>>>> 
>>>>>>> In general, using two sources of metadata is a lot more complex and
>>>>>>> error-prone than one.  A lot of the bugs and corner cases we have
>>> are
>>>>> the
>>>>>>> result of divergences between the controller and the state in
>>>>> ZooKeeper.
>>>>>>> Eliminating this divergence, and the split-brain scenarios it
>>> creates,
>>>>> is a
>>>>>>> major goal of this work.
>>>>>>> 
>>>>>>>> 
>>>>>>>> Regardless of whether what I wrote above has merit or not, I
>>> think
>>>>> the
>>>>>>> KIP
>>>>>>>> should be more explicit about what the upgrade constraints
>>> actually
>>>>> are.
>>>>>>>> Can the bridge release be adopted with Zookeeper remaining in
>>> place
>>>>> and
>>>>>>>> then cutting over as a second, follow-on step, or must the
>>> Controller
>>>>>>>> Quorum nodes be started first and the bridge release cannot be
>>> used
>>>>> with
>>>>>>>> Zookeeper at all?
>>>>>>> 
>>>>>>> As I mentioned above, the bridge release supports (indeed,
>>> requires)
>>>>>>> ZooKeeper.  I have added a little more text about this to KIP-500
>>> which
>>>>>>> hopefully makes it clearer.
>>>>>>> 
>>>>>>> best,
>>>>>>> Colin
>>>>>>> 
>>>>>>>> If the bridge release cannot be used with Zookeeper at
>>>>>>>> all, then no version at or beyond the bridge release is available
>>>>>>>> unless/until abandoning Zookeeper; if the bridge release can be
>>> used
>>>>> with
>>>>>>>> Zookeeper, then is it the only version that can be used with
>>>>> Zookeeper,
>>>>>>> or
>>>>>>>> can Zookeeper be kept for additional releases if desired?
>>>>>>>> 
>>>>>>>> Ron
>>>>>>>> 
>>>>>>>> On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <
>>> rndgstn@gmail.com>
>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Hi Colin.  The diagram up at the top confused me --
>>> specifically,
>>>>> the
>>>>>>>>> lines connecting the controller/active-controller to the
>>> brokers.
>>>>> I
>>>>>>> had
>>>>>>>>> assumed the arrows on those lines represented the direction of
>>> data
>>>>>>> flow,
>>>>>>>>> but that is not the case; the arrows actually identify the
>>> target
>>>>> of
>>>>>>> the
>>>>>>>>> action, and the non-arrowed end indicates the initiator of the
>>>>>>> action.  For
>>>>>>>>> example, the lines point from the controller to the brokers in
>>> the
>>>>>>> "today"
>>>>>>>>> section on the left to show that the controller pushes to the
>>>>> brokers;
>>>>>>> the
>>>>>>>>> lines point from the brokers to the active-controller in the
>>>>> "tomorrow"
>>>>>>>>> section on the right to show that the brokers pull from the
>>>>>>>>> active-controller.  As I said, this confused me because my gut
>>>>>>> instinct was
>>>>>>>>> to interpret the arrow as indicating the direction of data
>>> flow,
>>>>> and
>>>>>>> when I
>>>>>>>>> look at the "tomorrow" picture on the right I initially thought
>>>>>>> information
>>>>>>>>> was moving from the brokers to the active-controller.  Did you
>>>>> consider
>>>>>>>>> drawing that picture with the arrows reversed in the "tomorrow"
>>>>> side so
>>>>>>>>> that the arrows represent the direction of data flow, and then
>>> add
>>>>> the
>>>>>>>>> labels "push" on the "today" side and "pull" on the "tomorrow"
>>>>> side to
>>>>>>>>> indicate who initiates the data flow?  It occurs to me that
>>> this
>>>>>>> picture
>>>>>>>>> may end up being widely distributed, so it might be in
>>> everyone's
>>>>>>> interest
>>>>>>>>> to proactively avoid any possible confusion by being more
>>> explicit.
>>>>>>>>> 
>>>>>>>>> Minor corrections?
>>>>>>>>> <<<In the current world, a broker which can contact ZooKeeper
>>> but
>>>>> which
>>>>>>>>> is partitioned from the active controller
>>>>>>>>>>>> In the current world, a broker which can contact ZooKeeper
>>> but
>>>>> which
>>>>>>>>> is partitioned from the controller
>>>>>>>>> 
>>>>>>>>> <<<Eventually, the controller will ask the broker to finally go
>>>>> offline
>>>>>>>>>>>> Eventually, the active controller will ask the broker to
>>>>> finally go
>>>>>>>>> offline
>>>>>>>>> 
>>>>>>>>> <<<New versions of the clients should send these operations
>>>>> directly to
>>>>>>>>> the controller
>>>>>>>>>>>> New versions of the clients should send these operations
>>>>> directly to
>>>>>>>>> the active controller
>>>>>>>>> 
>>>>>>>>> <<<In the post-ZK world, the leader will make an RPC to the
>>>>> controller
>>>>>>>>> instead
>>>>>>>>>>>> In the post-ZK world, the leader will make an RPC to the
>>> active
>>>>>>>>> controller instead
>>>>>>>>> 
>>>>>>>>> <<<For example, the brokers may need to forward their requests
>>> to
>>>>> the
>>>>>>>>> controller.
>>>>>>>>>>>> For example, the brokers may need to forward their requests
>>> to
>>>>> the
>>>>>>>>> active controller.
>>>>>>>>> 
>>>>>>>>> <<<The new controller will monitor ZooKeeper for legacy broker
>>> node
>>>>>>>>> registrations
>>>>>>>>>>>> The new (active) controller will monitor ZooKeeper for
>>> legacy
>>>>> broker
>>>>>>>>> node registrations
>>>>>>>>> 
>>>>>>>>> Ron
>>>>>>>>> 
>>>>>>>>> On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <
>>> cmccabe@apache.org>
>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi all,
>>>>>>>>>> 
>>>>>>>>>> The KIP has been out for a while, so I'm thinking about
>>> calling a
>>>>> vote
>>>>>>>>>> some time this week.
>>>>>>>>>> 
>>>>>>>>>> best,
>>>>>>>>>> Colin
>>>>>>>>>> 
>>>>>>>>>>> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
>>>>>>>>>>>> On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
>>>>>>>>>>>> Thanks for the KIP, Colin. This looks great!
>>>>>>>>>>>> 
>>>>>>>>>>>> I really like the idea of separating the Controller and
>>> Broker
>>>>>>> JVMs.
>>>>>>>>>>>> 
>>>>>>>>>>>> As you alluded to above, it might be nice to have a
>>> separate
>>>>>>>>>>>> broker-registration API to avoid overloading the metadata
>>>>> fetch
>>>>>>> API.
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> Hi David,
>>>>>>>>>>> 
>>>>>>>>>>> Thanks for taking a look.
>>>>>>>>>>> 
>>>>>>>>>>> I removed the sentence about MetadataFetch also serving as
>>> the
>>>>>>> broker
>>>>>>>>>>> registration API.  I think I agree that we will probably
>>> want a
>>>>>>>>>>> separate RPC to fill this role.  We will have a follow-on
>>> KIP
>>>>> that
>>>>>>> will
>>>>>>>>>>> go into more detail about metadata propagation and
>>> registration
>>>>> in
>>>>>>> the
>>>>>>>>>>> post-ZK world.  That KIP will also have a full description
>>> of
>>>>> the
>>>>>>>>>>> registration RPC, etc.  For now, I think the important part
>>> for
>>>>>>> KIP-500
>>>>>>>>>>> is that the broker registers with the controller quorum.  On
>>>>>>>>>>> registration, the controller quorum assigns it a new broker
>>>>> epoch,
>>>>>>>>>>> which can distinguish successive broker incarnations.
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> When a broker gets a metadata delta, will it be a
>>> sequence of
>>>>>>> deltas
>>>>>>>>>> since
>>>>>>>>>>>> the last update or a cumulative delta since the last
>>> update?
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> It will be a sequence of deltas.  Basically, the broker
>>> will be
>>>>>>> reading
>>>>>>>>>>> from the metadata log.
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> Will we include any kind of integrity check on the deltas
>>> to
>>>>>>> ensure
>>>>>>>>>> the brokers
>>>>>>>>>>>> have applied them correctly? Perhaps this will be
>>> addressed in
>>>>>>> one of
>>>>>>>>>> the
>>>>>>>>>>>> follow-on KIPs.
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> In general, we will have checksums on the metadata that we
>>>>> fetch.
>>>>>>> This
>>>>>>>>>>> is similar to how we have checksums on regular data.  Or if
>>> the
>>>>>>>>>>> question is about catching logic errors in the metadata
>>> handling
>>>>>>> code,
>>>>>>>>>>> that sounds more like something that should be caught by
>>> test
>>>>> cases.
>>>>>>>>>>> 
>>>>>>>>>>> best,
>>>>>>>>>>> Colin
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> Thanks!
>>>>>>>>>>>> 
>>>>>>>>>>>> On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <
>>>>> cmccabe@apache.org>
>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi Mickael,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks for taking a look.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I don't think we want to support that kind of
>>> multi-tenancy
>>>>> at
>>>>>>> the
>>>>>>>>>>>>> controller level.  If the cluster is small enough that
>>> we
>>>>> want
>>>>>>> to
>>>>>>>>>> pack the
>>>>>>>>>>>>> controller(s) with something else, we could run them
>>>>> alongside
>>>>>>> the
>>>>>>>>>> brokers,
>>>>>>>>>>>>> or possibly inside three of the broker JVMs.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> best,
>>>>>>>>>>>>> Colin
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
>>>>>>>>>>>>>> Thank Colin for kickstarting this initiative.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Just one question.
>>>>>>>>>>>>>> - A nice feature of Zookeeper is the ability to use
>>>>> chroots
>>>>>>> and
>>>>>>>>>> have
>>>>>>>>>>>>>> several Kafka clusters use the same Zookeeper
>>> ensemble. Is
>>>>>>> this
>>>>>>>>>>>>>> something we should keep?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
>>>>>>> cmccabe@apache.org>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
>>>>>>>>>>>>>>>> Hi Colin,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Currently ZooKeeper provides a convenient
>>> notification
>>>>>>>>>> mechanism for
>>>>>>>>>>>>>>>> knowing that broker and topic configuration has
>>>>> changed.
>>>>>>> While
>>>>>>>>>>>>> KIP-500 does
>>>>>>>>>>>>>>>> suggest that incremental metadata update is
>>> expected
>>>>> to
>>>>>>> come
>>>>>>>>>> to
>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>> eventually, that would seem to imply that for some
>>>>> number
>>>>>>> of
>>>>>>>>>>>>> releases there
>>>>>>>>>>>>>>>> would be no equivalent mechanism for knowing about
>>>>> config
>>>>>>>>>> changes.
>>>>>>>>>>>>> Is there
>>>>>>>>>>>>>>>> any thinking at this point about how a similar
>>>>>>> notification
>>>>>>>>>> might be
>>>>>>>>>>>>>>>> provided in the future?
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> We could eventually have some inotify-like mechanism
>>>>> where
>>>>>>>>>> clients
>>>>>>>>>>>>> could register interest in various types of events and
>>> got
>>>>>>> notified
>>>>>>>>>> when
>>>>>>>>>>>>> they happened.  Reading the metadata log is conceptually
>>>>> simple.
>>>>>>>>>> The main
>>>>>>>>>>>>> complexity would be in setting up an API that made
>>> sense and
>>>>>>> that
>>>>>>>>>> didn't
>>>>>>>>>>>>> unduly constrain future implementations.  We'd have to
>>> think
>>>>>>>>>> carefully
>>>>>>>>>>>>> about what the real use-cases for this were, though.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Tom
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Mon, Aug 5, 2019 at 3:49 PM Viktor
>>> Somogyi-Vass <
>>>>>>>>>>>>> viktorsomogyi@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Hey Colin,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> I think this is a long-awaited KIP, thanks for
>>>>> driving
>>>>>>> it.
>>>>>>>>>> I'm
>>>>>>>>>>>>> excited to
>>>>>>>>>>>>>>>>> see this in Kafka once. I collected my questions
>>>>> (and I
>>>>>>>>>> accept the
>>>>>>>>>>>>> "TBD"
>>>>>>>>>>>>>>>>> answer as they might be a bit deep for this high
>>>>> level
>>>>>>> :) ).
>>>>>>>>>>>>>>>>> 1.) Are there any specific reasons for the
>>>>> Controller
>>>>>>> just
>>>>>>>>>>>>> periodically
>>>>>>>>>>>>>>>>> persisting its state on disk periodically
>>> instead of
>>>>>>>>>>>>> asynchronously with
>>>>>>>>>>>>>>>>> every update? Wouldn't less frequent saves
>>> increase
>>>>> the
>>>>>>>>>> chance for
>>>>>>>>>>>>> missing
>>>>>>>>>>>>>>>>> a state change if the controller crashes
>>> between two
>>>>>>> saves?
>>>>>>>>>>>>>>>>> 2.) Why can't we allow brokers to fetch metadata
>>>>> from
>>>>>>> the
>>>>>>>>>> follower
>>>>>>>>>>>>>>>>> controllers? I assume that followers would have
>>>>>>> up-to-date
>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>>> therefore brokers could fetch from there in
>>> theory.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Viktor
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
>>>>>>>>>>>>> reluctanthero104@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Thanks for explaining Ismael! Breaking down
>>> into
>>>>>>>>>> follow-up KIPs
>>>>>>>>>>>>> sounds
>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>> a good idea.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
>>>>>>>>>> ismael@juma.me.uk>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Hi Boyang,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Yes, there will be several KIPs that will
>>>>> discuss
>>>>>>> the
>>>>>>>>>> items you
>>>>>>>>>>>>>>>>> describe
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> detail. Colin, it may be helpful to make
>>> this
>>>>> clear
>>>>>>> in
>>>>>>>>>> the KIP
>>>>>>>>>>>>> 500
>>>>>>>>>>>>>>>>>>> description.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
>>>>>>>>>>>>> reluctanthero104@gmail.com
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Thanks Colin for initiating this important
>>>>> effort!
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> One question I have is whether we have a
>>>>> session
>>>>>>>>>> discussing
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>>>> failover in the new architecture? I know
>>> we
>>>>> are
>>>>>>> using
>>>>>>>>>> Raft
>>>>>>>>>>>>> protocol
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> failover, yet it's still valuable to
>>> discuss
>>>>> the
>>>>>>>>>> steps new
>>>>>>>>>>>>> cluster is
>>>>>>>>>>>>>>>>>>> going
>>>>>>>>>>>>>>>>>>>> to take to reach the stable stage again,
>>> so
>>>>> that
>>>>>>> we
>>>>>>>>>> could
>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>> measure
>>>>>>>>>>>>>>>>>>>> the availability of the metadata servers.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Another suggestion I have is to write a
>>>>>>> step-by-step
>>>>>>>>>> design
>>>>>>>>>>>>> doc like
>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>> we did in KIP-98
>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>> 
>>>>> 
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>>>>>>>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>> including the new request protocols and
>>> how
>>>>> they
>>>>>>> are
>>>>>>>>>>>>> interacting in
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>> cluster. For a complicated change like
>>> this,
>>>>> an
>>>>>>>>>>>>> implementation design
>>>>>>>>>>>>>>>>>> doc
>>>>>>>>>>>>>>>>>>>> help a lot in the review process,
>>> otherwise
>>>>> most
>>>>>>>>>> discussions
>>>>>>>>>>>>> we have
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> focus on high level and lose important
>>>>> details as
>>>>>>> we
>>>>>>>>>>>>> discover them in
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> post-agreement phase.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Boyang
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> On Fri, Aug 2, 2019 at 5:17 PM Colin
>>> McCabe <
>>>>>>>>>>>>> cmccabe@apache.org>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 2, 2019, at 16:33, Jose
>>> Armando
>>>>>>> Garcia
>>>>>>>>>> Sancio
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> Thanks Colin for the detail KIP. I
>>> have a
>>>>> few
>>>>>>>>>> comments
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> questions.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> In the KIP's Motivation and Overview
>>> you
>>>>>>>>>> mentioned the
>>>>>>>>>>>>>>>>> LeaderAndIsr
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> UpdateMetadata RPC. For example,
>>> "updates
>>>>>>> which
>>>>>>>>>> the
>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>>> pushes,
>>>>>>>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>>>>>>> as LeaderAndIsr and UpdateMetadata
>>>>> messages".
>>>>>>> Is
>>>>>>>>>> your
>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> use MetadataFetch as a replacement to
>>> just
>>>>>>>>>>>>> UpdateMetadata only
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>> topic configuration in this state?
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Hi Jose,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Thanks for taking a look.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> The goal is for MetadataFetchRequest to
>>>>> replace
>>>>>>> both
>>>>>>>>>>>>>>>>>>> LeaderAndIsrRequest
>>>>>>>>>>>>>>>>>>>>> and UpdateMetadataRequest.  Topic
>>>>> configurations
>>>>>>>>>> would be
>>>>>>>>>>>>> fetched
>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>> with the other metadata.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> In the section "Broker Metadata
>>>>> Management",
>>>>>>> you
>>>>>>>>>> mention
>>>>>>>>>>>>> "Just
>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> fetch request, the broker will track
>>> the
>>>>>>> offset
>>>>>>>>>> of the
>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>> updates
>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>> fetched". To keep the log consistent
>>> Raft
>>>>>>>>>> requires that
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> followers
>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>> all of the log entries (term/epoch and
>>>>> offset)
>>>>>>>>>> that are
>>>>>>>>>>>>> after the
>>>>>>>>>>>>>>>>>>>>>> highwatermark. Any log entry before
>>> the
>>>>>>>>>> highwatermark
>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>> compacted/snapshot. Do we expect the
>>>>>>>>>> MetadataFetch API
>>>>>>>>>>>>> to only
>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>> entries up to the highwatermark?
>>> Unlike
>>>>> the
>>>>>>> Raft
>>>>>>>>>>>>> replication API
>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>> will replicate/fetch log entries
>>> after the
>>>>>>>>>> highwatermark
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> consensus?
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Good question.  Clearly, we shouldn't
>>> expose
>>>>>>>>>> metadata
>>>>>>>>>>>>> updates to
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> brokers until they've been stored on a
>>>>> majority
>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>> Raft nodes.
>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>> most obvious way to do that, like you
>>>>>>> mentioned, is
>>>>>>>>>> to
>>>>>>>>>>>>> have the
>>>>>>>>>>>>>>>>>> brokers
>>>>>>>>>>>>>>>>>>>>> only fetch up to the HWM, but not
>>> beyond.
>>>>> There
>>>>>>>>>> might be
>>>>>>>>>>>>> a more
>>>>>>>>>>>>>>>>>> clever
>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>> to do it by fetching the data, but not
>>>>> having
>>>>>>> the
>>>>>>>>>> brokers
>>>>>>>>>>>>> act on it
>>>>>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>> the HWM advances.  I'm not sure if
>>> that's
>>>>> worth
>>>>>>> it
>>>>>>>>>> or
>>>>>>>>>>>>> not.  We'll
>>>>>>>>>>>>>>>>>>> discuss
>>>>>>>>>>>>>>>>>>>>> this more in a separate KIP that just
>>>>> discusses
>>>>>>>>>> just Raft.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> In section "Broker Metadata
>>> Management",
>>>>> you
>>>>>>>>>> mention "the
>>>>>>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> send a full metadata image rather
>>> than a
>>>>>>> series of
>>>>>>>>>>>>> deltas". This
>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>> doesn't go into the set of operations
>>> that
>>>>>>> need
>>>>>>>>>> to be
>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>> top
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> Raft but it would be interested if
>>> this
>>>>> "full
>>>>>>>>>> metadata
>>>>>>>>>>>>> image"
>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> express also as deltas. For example,
>>>>> assuming
>>>>>>> we
>>>>>>>>>> are
>>>>>>>>>>>>> replicating
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> "full metadata image" could be a
>>> sequence
>>>>> of
>>>>>>> "put"
>>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>>> (znode
>>>>>>>>>>>>>>>>>>>>> create
>>>>>>>>>>>>>>>>>>>>>> to borrow ZK semantics).
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> The full image can definitely be
>>> expressed
>>>>> as a
>>>>>>> sum
>>>>>>>>>> of
>>>>>>>>>>>>> deltas.  At
>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>> point, the number of deltas will get
>>> large
>>>>>>> enough
>>>>>>>>>> that
>>>>>>>>>>>>> sending a
>>>>>>>>>>>>>>>>> full
>>>>>>>>>>>>>>>>>>>> image
>>>>>>>>>>>>>>>>>>>>> is better, though.  One question that
>>> we're
>>>>>>> still
>>>>>>>>>> thinking
>>>>>>>>>>>>> about is
>>>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>>> much of this can be shared with generic
>>>>> Kafka
>>>>>>> log
>>>>>>>>>> code,
>>>>>>>>>>>>> and how
>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>> be different.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> In section "Broker Metadata
>>> Management",
>>>>> you
>>>>>>>>>> mention
>>>>>>>>>>>>> "This
>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> double as a heartbeat, letting the
>>>>> controller
>>>>>>>>>> know that
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> alive". In section "Broker State
>>>>> Machine", you
>>>>>>>>>> mention
>>>>>>>>>>>>> "The
>>>>>>>>>>>>>>>>>>>> MetadataFetch
>>>>>>>>>>>>>>>>>>>>>> API serves as this registration
>>>>> mechanism".
>>>>>>> Does
>>>>>>>>>> this
>>>>>>>>>>>>> mean that
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> MetadataFetch Request will optionally
>>>>> include
>>>>>>>>>> broker
>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>> information?
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> I was originally thinking that the
>>>>>>>>>> MetadataFetchRequest
>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>> broker configuration information.
>>> Thinking
>>>>>>> about
>>>>>>>>>> this
>>>>>>>>>>>>> more, maybe
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> should just have a special registration
>>> RPC
>>>>> that
>>>>>>>>>> contains
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> information,
>>>>>>>>>>>>>>>>>>>>> to avoid sending it over the wire all
>>> the
>>>>> time.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Does this also mean that MetadataFetch
>>>>> request
>>>>>>>>>> will
>>>>>>>>>>>>> result in
>>>>>>>>>>>>>>>>>>>>>> a "write"/AppendEntries through the
>>> Raft
>>>>>>>>>> replication
>>>>>>>>>>>>> protocol
>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> can send the associated MetadataFetch
>>>>>>> Response?
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> I think we should require the broker to
>>> be
>>>>> out
>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>> Offline state
>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>> allowing it to fetch metadata, yes.  So
>>> the
>>>>>>> separate
>>>>>>>>>>>>> registration
>>>>>>>>>>>>>>>>> RPC
>>>>>>>>>>>>>>>>>>>>> should have completed first.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> In section "Broker State", you mention
>>>>> that a
>>>>>>>>>> broker can
>>>>>>>>>>>>>>>>> transition
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> online after it is caught with the
>>>>> metadata.
>>>>>>> What
>>>>>>>>>> do you
>>>>>>>>>>>>> mean by
>>>>>>>>>>>>>>>>>>> this?
>>>>>>>>>>>>>>>>>>>>>> Metadata is always changing. How does
>>> the
>>>>>>> broker
>>>>>>>>>> know
>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>> caught
>>>>>>>>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>>>>>>>>> since it doesn't participate in the
>>>>> consensus
>>>>>>> or
>>>>>>>>>> the
>>>>>>>>>>>>> advancement
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> highwatermark?
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> That's a good point.  Being "caught up"
>>> is
>>>>>>> somewhat
>>>>>>>>>> of a
>>>>>>>>>>>>> fuzzy
>>>>>>>>>>>>>>>>>> concept
>>>>>>>>>>>>>>>>>>>>> here, since the brokers do not
>>> participate
>>>>> in
>>>>>>> the
>>>>>>>>>> metadata
>>>>>>>>>>>>>>>>> consensus.
>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>> think ideally we would want to define
>>> it in
>>>>>>> terms
>>>>>>>>>> of time
>>>>>>>>>>>>> ("the
>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>> all the updates from the last 2
>>> minutes",
>>>>> for
>>>>>>>>>> example.)
>>>>>>>>>>>>> We should
>>>>>>>>>>>>>>>>>>> spell
>>>>>>>>>>>>>>>>>>>>> this out better in the KIP.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> In section "Start the controller
>>> quorum
>>>>>>> nodes",
>>>>>>>>>> you
>>>>>>>>>>>>> mention "Once
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>> taken over the /controller node, the
>>>>> active
>>>>>>>>>> controller
>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>> proceed
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> load
>>>>>>>>>>>>>>>>>>>>>> the full state of ZooKeeper.  It will
>>>>> write
>>>>>>> out
>>>>>>>>>> this
>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> quorum's metadata storage.  After this
>>>>> point,
>>>>>>> the
>>>>>>>>>>>>> metadata quorum
>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> the metadata store of record, rather
>>> than
>>>>> the
>>>>>>>>>> data in
>>>>>>>>>>>>> ZooKeeper."
>>>>>>>>>>>>>>>>>>>> During
>>>>>>>>>>>>>>>>>>>>>> this migration do should we expect to
>>>>> have a
>>>>>>>>>> small period
>>>>>>>>>>>>>>>>>> controller
>>>>>>>>>>>>>>>>>>>>>> unavailability while the controller
>>>>> replicas
>>>>>>> this
>>>>>>>>>> state
>>>>>>>>>>>>> to all of
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> raft
>>>>>>>>>>>>>>>>>>>>>> nodes in the controller quorum and we
>>>>> buffer
>>>>>>> new
>>>>>>>>>>>>> controller API
>>>>>>>>>>>>>>>>>>>> requests?
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Yes, the controller would be unavailable
>>>>> during
>>>>>>> this
>>>>>>>>>>>>> time.  I don't
>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>> this will be that different from the
>>> current
>>>>>>> period
>>>>>>>>>> of
>>>>>>>>>>>>>>>>> unavailability
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> a new controller starts up and needs to
>>>>> load the
>>>>>>>>>> full
>>>>>>>>>>>>> state from
>>>>>>>>>>>>>>>>> ZK.
>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>> main difference is that in this period,
>>> we'd
>>>>>>> have
>>>>>>>>>> to write
>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>> controller quorum rather than just to
>>>>> memory.
>>>>>>> But
>>>>>>>>>> we
>>>>>>>>>>>>> believe this
>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>> be pretty fast.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> regards,
>>>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>> -Jose
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> --
>>>>>>>>>>>> David Arthur
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> 


Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

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

Good point.  I added a section titled "future work" with information about the follow-on KIPs that we discussed here.

best,
Colin


On Fri, Aug 23, 2019, at 13:15, Ryanne Dolan wrote:
> Thanks Colin, sgtm. Please make this clear in the KIP -- otherwise it is
> hard to nail down what we are voting for.
> 
> Ryanne
> 
> 
> On Fri, Aug 23, 2019, 12:58 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > On Fri, Aug 23, 2019, at 06:24, Ryanne Dolan wrote:
> > > Colin, can you outline what specifically would be in scope for this KIP
> > vs
> > > deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
> > > section? Is the idea to get to the bridge release with this KIP, and then
> > > go from there?
> > >
> > > Ryanne
> > >
> >
> > Hi Ryanne,
> >
> > The goal for KIP-500 is to set out an overall vision for how we will
> > remove ZooKeeper and transition to managing metadata via a controller
> > quorum.
> >
> > We will create follow-on KIPs that will lay out the specific details of
> > each step.
> >
> > * A KIP for allowing kafka-configs.sh to change topic configurations
> > without using ZooKeeper.  (It can already change broker configurations
> > without ZK)
> >
> > * A KIP for adding APIs to replace direct ZK access by the brokers.
> >
> > * A KIP to describe Raft replication in Kafka, including the overall
> > protocol, details of each RPC, etc.
> >
> > * A KIP describing the controller changes, how metadata is stored, etc.
> >
> > There may be other KIPs that we need (for example, if we find another tool
> > that still has a hard ZK dependency), but that's the general idea.  KIP-500
> > is about the overall design-- the follow on KIPs are about the specific
> > details.
> >
> > best,
> > Colin
> >
> >
> > >
> > > On Thu, Aug 22, 2019, 11:58 AM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> > > > > Thanks, Colin.  The changes you made to the KIP related to the bridge
> > > > > release help make it clearer.  I still have some confusion about the
> > > > phrase
> > > > > "The rolling upgrade from the bridge release will take several
> > steps."
> > > > > This made me think you are talking about moving from the bridge
> > release
> > > > to
> > > > > some other, newer, release that comes after the bridge release.  But
> > I
> > > > > think what you are getting at is that the bridge release can be run
> > with
> > > > or
> > > > > without Zookeeper -- when first upgrading to it Zookeeper remains in
> > use,
> > > > > but then there is a transition that can be made to engage the warp
> > > > drive...
> > > > > I mean the Controller Quorum.  So maybe the phrase should be "The
> > rolling
> > > > > upgrade through the bridge release -- starting with Zookeeper being
> > in
> > > > use
> > > > > and ending with Zookeeper having been replaced by the Controller
> > Quorum
> > > > --
> > > > > will take several steps."
> > > >
> > > > Hi Ron,
> > > >
> > > > To clarify, the bridge release will require ZooKeeper.  It will also
> > not
> > > > support the controller quorum.  It's a bridge in the sense that you
> > must
> > > > upgrade to a bridge release prior to upgrading to a ZK-less release.  I
> > > > added some more descriptive text to the bridge release paragraph--
> > > > hopefully this makes it clearer.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Do I understand it correctly, and might some change in phrasing or
> > > > > additional clarification help others avoid the same confusion I had?
> > > > >
> > > > > Ron
> > > > >
> > > > > On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > >
> > > > > > On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> > > > > > > Hi Colin.  I like the concept of a "bridge release" for migrating
> > > > off of
> > > > > > > Zookeeper, but I worry that it may become a bottleneck if people
> > > > hesitate
> > > > > > > to replace Zookeeper -- they would be unable to adopt newer
> > versions
> > > > of
> > > > > > > Kafka until taking (what feels to them like) a giant leap.  As an
> > > > > > example,
> > > > > > > assuming version 4.0.x of Kafka is the supported bridge release,
> > I
> > > > would
> > > > > > > not be surprised if uptake of the 4.x release and the time-based
> > > > releases
> > > > > > > that follow it end up being much slower due to the perceived
> > barrier.
> > > > > > >
> > > > > > > Any perceived barrier could be lowered if the 4.0.x release could
> > > > > > > optionally continue to use Zookeeper -- then the cutover would
> > be two
> > > > > > > incremental steps (move to 4.0.x, then replace Zookeeper while
> > > > staying on
> > > > > > > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and
> > replace
> > > > > > > Zookeeper in one fell swoop).
> > > > > >
> > > > > > Hi Ron,
> > > > > >
> > > > > > Just to clarify, the "bridge release" will continue to use
> > ZooKeeper.
> > > > It
> > > > > > will not support running without ZooKeeper.  It is the releases
> > that
> > > > follow
> > > > > > the bridge release that will remove ZooKeeper.
> > > > > >
> > > > > > Also, it's a bit unclear whether the bridge release would be 3.x or
> > > > 4.x,
> > > > > > or something to follow.  We do know that the bridge release can't
> > be a
> > > > 2.x
> > > > > > release, since it requires at least one incompatible change,
> > removing
> > > > > > --zookeeper options from all the shell scripts.  (Since we're doing
> > > > > > semantic versioning, any time we make an incompatible change, we
> > bump
> > > > the
> > > > > > major version number.)
> > > > > >
> > > > > > In general, using two sources of metadata is a lot more complex and
> > > > > > error-prone than one.  A lot of the bugs and corner cases we have
> > are
> > > > the
> > > > > > result of divergences between the controller and the state in
> > > > ZooKeeper.
> > > > > > Eliminating this divergence, and the split-brain scenarios it
> > creates,
> > > > is a
> > > > > > major goal of this work.
> > > > > >
> > > > > > >
> > > > > > > Regardless of whether what I wrote above has merit or not, I
> > think
> > > > the
> > > > > > KIP
> > > > > > > should be more explicit about what the upgrade constraints
> > actually
> > > > are.
> > > > > > > Can the bridge release be adopted with Zookeeper remaining in
> > place
> > > > and
> > > > > > > then cutting over as a second, follow-on step, or must the
> > Controller
> > > > > > > Quorum nodes be started first and the bridge release cannot be
> > used
> > > > with
> > > > > > > Zookeeper at all?
> > > > > >
> > > > > > As I mentioned above, the bridge release supports (indeed,
> > requires)
> > > > > > ZooKeeper.  I have added a little more text about this to KIP-500
> > which
> > > > > > hopefully makes it clearer.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > >  If the bridge release cannot be used with Zookeeper at
> > > > > > > all, then no version at or beyond the bridge release is available
> > > > > > > unless/until abandoning Zookeeper; if the bridge release can be
> > used
> > > > with
> > > > > > > Zookeeper, then is it the only version that can be used with
> > > > Zookeeper,
> > > > > > or
> > > > > > > can Zookeeper be kept for additional releases if desired?
> > > > > > >
> > > > > > > Ron
> > > > > > >
> > > > > > > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <
> > rndgstn@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Colin.  The diagram up at the top confused me --
> > specifically,
> > > > the
> > > > > > > > lines connecting the controller/active-controller to the
> > brokers.
> > > > I
> > > > > > had
> > > > > > > > assumed the arrows on those lines represented the direction of
> > data
> > > > > > flow,
> > > > > > > > but that is not the case; the arrows actually identify the
> > target
> > > > of
> > > > > > the
> > > > > > > > action, and the non-arrowed end indicates the initiator of the
> > > > > > action.  For
> > > > > > > > example, the lines point from the controller to the brokers in
> > the
> > > > > > "today"
> > > > > > > > section on the left to show that the controller pushes to the
> > > > brokers;
> > > > > > the
> > > > > > > > lines point from the brokers to the active-controller in the
> > > > "tomorrow"
> > > > > > > > section on the right to show that the brokers pull from the
> > > > > > > > active-controller.  As I said, this confused me because my gut
> > > > > > instinct was
> > > > > > > > to interpret the arrow as indicating the direction of data
> > flow,
> > > > and
> > > > > > when I
> > > > > > > > look at the "tomorrow" picture on the right I initially thought
> > > > > > information
> > > > > > > > was moving from the brokers to the active-controller.  Did you
> > > > consider
> > > > > > > > drawing that picture with the arrows reversed in the "tomorrow"
> > > > side so
> > > > > > > > that the arrows represent the direction of data flow, and then
> > add
> > > > the
> > > > > > > > labels "push" on the "today" side and "pull" on the "tomorrow"
> > > > side to
> > > > > > > > indicate who initiates the data flow?  It occurs to me that
> > this
> > > > > > picture
> > > > > > > > may end up being widely distributed, so it might be in
> > everyone's
> > > > > > interest
> > > > > > > > to proactively avoid any possible confusion by being more
> > explicit.
> > > > > > > >
> > > > > > > > Minor corrections?
> > > > > > > > <<<In the current world, a broker which can contact ZooKeeper
> > but
> > > > which
> > > > > > > > is partitioned from the active controller
> > > > > > > > >>>In the current world, a broker which can contact ZooKeeper
> > but
> > > > which
> > > > > > > > is partitioned from the controller
> > > > > > > >
> > > > > > > > <<<Eventually, the controller will ask the broker to finally go
> > > > offline
> > > > > > > > >>>Eventually, the active controller will ask the broker to
> > > > finally go
> > > > > > > > offline
> > > > > > > >
> > > > > > > > <<<New versions of the clients should send these operations
> > > > directly to
> > > > > > > > the controller
> > > > > > > > >>>New versions of the clients should send these operations
> > > > directly to
> > > > > > > > the active controller
> > > > > > > >
> > > > > > > > <<<In the post-ZK world, the leader will make an RPC to the
> > > > controller
> > > > > > > > instead
> > > > > > > > >>>In the post-ZK world, the leader will make an RPC to the
> > active
> > > > > > > > controller instead
> > > > > > > >
> > > > > > > > <<<For example, the brokers may need to forward their requests
> > to
> > > > the
> > > > > > > > controller.
> > > > > > > > >>>For example, the brokers may need to forward their requests
> > to
> > > > the
> > > > > > > > active controller.
> > > > > > > >
> > > > > > > > <<<The new controller will monitor ZooKeeper for legacy broker
> > node
> > > > > > > > registrations
> > > > > > > > >>>The new (active) controller will monitor ZooKeeper for
> > legacy
> > > > broker
> > > > > > > > node registrations
> > > > > > > >
> > > > > > > > Ron
> > > > > > > >
> > > > > > > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <
> > cmccabe@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > >> Hi all,
> > > > > > > >>
> > > > > > > >> The KIP has been out for a while, so I'm thinking about
> > calling a
> > > > vote
> > > > > > > >> some time this week.
> > > > > > > >>
> > > > > > > >> best,
> > > > > > > >> Colin
> > > > > > > >>
> > > > > > > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > > > > > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > > > > > >> > > Thanks for the KIP, Colin. This looks great!
> > > > > > > >> > >
> > > > > > > >> > > I really like the idea of separating the Controller and
> > Broker
> > > > > > JVMs.
> > > > > > > >> > >
> > > > > > > >> > > As you alluded to above, it might be nice to have a
> > separate
> > > > > > > >> > > broker-registration API to avoid overloading the metadata
> > > > fetch
> > > > > > API.
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > Hi David,
> > > > > > > >> >
> > > > > > > >> > Thanks for taking a look.
> > > > > > > >> >
> > > > > > > >> > I removed the sentence about MetadataFetch also serving as
> > the
> > > > > > broker
> > > > > > > >> > registration API.  I think I agree that we will probably
> > want a
> > > > > > > >> > separate RPC to fill this role.  We will have a follow-on
> > KIP
> > > > that
> > > > > > will
> > > > > > > >> > go into more detail about metadata propagation and
> > registration
> > > > in
> > > > > > the
> > > > > > > >> > post-ZK world.  That KIP will also have a full description
> > of
> > > > the
> > > > > > > >> > registration RPC, etc.  For now, I think the important part
> > for
> > > > > > KIP-500
> > > > > > > >> > is that the broker registers with the controller quorum.  On
> > > > > > > >> > registration, the controller quorum assigns it a new broker
> > > > epoch,
> > > > > > > >> > which can distinguish successive broker incarnations.
> > > > > > > >> >
> > > > > > > >> > >
> > > > > > > >> > > When a broker gets a metadata delta, will it be a
> > sequence of
> > > > > > deltas
> > > > > > > >> since
> > > > > > > >> > > the last update or a cumulative delta since the last
> > update?
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > It will be a sequence of deltas.  Basically, the broker
> > will be
> > > > > > reading
> > > > > > > >> > from the metadata log.
> > > > > > > >> >
> > > > > > > >> > >
> > > > > > > >> > > Will we include any kind of integrity check on the deltas
> > to
> > > > > > ensure
> > > > > > > >> the brokers
> > > > > > > >> > > have applied them correctly? Perhaps this will be
> > addressed in
> > > > > > one of
> > > > > > > >> the
> > > > > > > >> > > follow-on KIPs.
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > In general, we will have checksums on the metadata that we
> > > > fetch.
> > > > > > This
> > > > > > > >> > is similar to how we have checksums on regular data.  Or if
> > the
> > > > > > > >> > question is about catching logic errors in the metadata
> > handling
> > > > > > code,
> > > > > > > >> > that sounds more like something that should be caught by
> > test
> > > > cases.
> > > > > > > >> >
> > > > > > > >> > best,
> > > > > > > >> > Colin
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > > Thanks!
> > > > > > > >> > >
> > > > > > > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <
> > > > cmccabe@apache.org>
> > > > > > > >> wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hi Mickael,
> > > > > > > >> > > >
> > > > > > > >> > > > Thanks for taking a look.
> > > > > > > >> > > >
> > > > > > > >> > > > I don't think we want to support that kind of
> > multi-tenancy
> > > > at
> > > > > > the
> > > > > > > >> > > > controller level.  If the cluster is small enough that
> > we
> > > > want
> > > > > > to
> > > > > > > >> pack the
> > > > > > > >> > > > controller(s) with something else, we could run them
> > > > alongside
> > > > > > the
> > > > > > > >> brokers,
> > > > > > > >> > > > or possibly inside three of the broker JVMs.
> > > > > > > >> > > >
> > > > > > > >> > > > best,
> > > > > > > >> > > > Colin
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > > > > >> > > > > Thank Colin for kickstarting this initiative.
> > > > > > > >> > > > >
> > > > > > > >> > > > > Just one question.
> > > > > > > >> > > > > - A nice feature of Zookeeper is the ability to use
> > > > chroots
> > > > > > and
> > > > > > > >> have
> > > > > > > >> > > > > several Kafka clusters use the same Zookeeper
> > ensemble. Is
> > > > > > this
> > > > > > > >> > > > > something we should keep?
> > > > > > > >> > > > >
> > > > > > > >> > > > > Thanks
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> > > > > > cmccabe@apache.org>
> > > > > > > >> wrote:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > > > > >> > > > > > > Hi Colin,
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Thanks for the KIP.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Currently ZooKeeper provides a convenient
> > notification
> > > > > > > >> mechanism for
> > > > > > > >> > > > > > > knowing that broker and topic configuration has
> > > > changed.
> > > > > > While
> > > > > > > >> > > > KIP-500 does
> > > > > > > >> > > > > > > suggest that incremental metadata update is
> > expected
> > > > to
> > > > > > come
> > > > > > > >> to
> > > > > > > >> > > > clients
> > > > > > > >> > > > > > > eventually, that would seem to imply that for some
> > > > number
> > > > > > of
> > > > > > > >> > > > releases there
> > > > > > > >> > > > > > > would be no equivalent mechanism for knowing about
> > > > config
> > > > > > > >> changes.
> > > > > > > >> > > > Is there
> > > > > > > >> > > > > > > any thinking at this point about how a similar
> > > > > > notification
> > > > > > > >> might be
> > > > > > > >> > > > > > > provided in the future?
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > We could eventually have some inotify-like mechanism
> > > > where
> > > > > > > >> clients
> > > > > > > >> > > > could register interest in various types of events and
> > got
> > > > > > notified
> > > > > > > >> when
> > > > > > > >> > > > they happened.  Reading the metadata log is conceptually
> > > > simple.
> > > > > > > >> The main
> > > > > > > >> > > > complexity would be in setting up an API that made
> > sense and
> > > > > > that
> > > > > > > >> didn't
> > > > > > > >> > > > unduly constrain future implementations.  We'd have to
> > think
> > > > > > > >> carefully
> > > > > > > >> > > > about what the real use-cases for this were, though.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > best,
> > > > > > > >> > > > > > Colin
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Thanks,
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Tom
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor
> > Somogyi-Vass <
> > > > > > > >> > > > viktorsomogyi@gmail.com>
> > > > > > > >> > > > > > > wrote:
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > Hey Colin,
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > I think this is a long-awaited KIP, thanks for
> > > > driving
> > > > > > it.
> > > > > > > >> I'm
> > > > > > > >> > > > excited to
> > > > > > > >> > > > > > > > see this in Kafka once. I collected my questions
> > > > (and I
> > > > > > > >> accept the
> > > > > > > >> > > > "TBD"
> > > > > > > >> > > > > > > > answer as they might be a bit deep for this high
> > > > level
> > > > > > :) ).
> > > > > > > >> > > > > > > > 1.) Are there any specific reasons for the
> > > > Controller
> > > > > > just
> > > > > > > >> > > > periodically
> > > > > > > >> > > > > > > > persisting its state on disk periodically
> > instead of
> > > > > > > >> > > > asynchronously with
> > > > > > > >> > > > > > > > every update? Wouldn't less frequent saves
> > increase
> > > > the
> > > > > > > >> chance for
> > > > > > > >> > > > missing
> > > > > > > >> > > > > > > > a state change if the controller crashes
> > between two
> > > > > > saves?
> > > > > > > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata
> > > > from
> > > > > > the
> > > > > > > >> follower
> > > > > > > >> > > > > > > > controllers? I assume that followers would have
> > > > > > up-to-date
> > > > > > > >> > > > information
> > > > > > > >> > > > > > > > therefore brokers could fetch from there in
> > theory.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Thanks,
> > > > > > > >> > > > > > > > Viktor
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > > > > > >> > > > reluctanthero104@gmail.com>
> > > > > > > >> > > > > > > > wrote:
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > > Thanks for explaining Ismael! Breaking down
> > into
> > > > > > > >> follow-up KIPs
> > > > > > > >> > > > sounds
> > > > > > > >> > > > > > > > like
> > > > > > > >> > > > > > > > > a good idea.
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > > > > > > >> ismael@juma.me.uk>
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > > Hi Boyang,
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > > > Yes, there will be several KIPs that will
> > > > discuss
> > > > > > the
> > > > > > > >> items you
> > > > > > > >> > > > > > > > describe
> > > > > > > >> > > > > > > > > in
> > > > > > > >> > > > > > > > > > detail. Colin, it may be helpful to make
> > this
> > > > clear
> > > > > > in
> > > > > > > >> the KIP
> > > > > > > >> > > > 500
> > > > > > > >> > > > > > > > > > description.
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > > > Ismael
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > > > > > >> > > > reluctanthero104@gmail.com
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > > wrote:
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > > > > Thanks Colin for initiating this important
> > > > effort!
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > One question I have is whether we have a
> > > > session
> > > > > > > >> discussing
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > > > controller
> > > > > > > >> > > > > > > > > > > failover in the new architecture? I know
> > we
> > > > are
> > > > > > using
> > > > > > > >> Raft
> > > > > > > >> > > > protocol
> > > > > > > >> > > > > > > > to
> > > > > > > >> > > > > > > > > > > failover, yet it's still valuable to
> > discuss
> > > > the
> > > > > > > >> steps new
> > > > > > > >> > > > cluster is
> > > > > > > >> > > > > > > > > > going
> > > > > > > >> > > > > > > > > > > to take to reach the stable stage again,
> > so
> > > > that
> > > > > > we
> > > > > > > >> could
> > > > > > > >> > > > easily
> > > > > > > >> > > > > > > > > measure
> > > > > > > >> > > > > > > > > > > the availability of the metadata servers.
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > Another suggestion I have is to write a
> > > > > > step-by-step
> > > > > > > >> design
> > > > > > > >> > > > doc like
> > > > > > > >> > > > > > > > > what
> > > > > > > >> > > > > > > > > > > we did in KIP-98
> > > > > > > >> > > > > > > > > > > <
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > >
> > > > > > > >> > > >
> > > > > > > >>
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > >> > > > > > > > > > > >,
> > > > > > > >> > > > > > > > > > > including the new request protocols and
> > how
> > > > they
> > > > > > are
> > > > > > > >> > > > interacting in
> > > > > > > >> > > > > > > > the
> > > > > > > >> > > > > > > > > > new
> > > > > > > >> > > > > > > > > > > cluster. For a complicated change like
> > this,
> > > > an
> > > > > > > >> > > > implementation design
> > > > > > > >> > > > > > > > > doc
> > > > > > > >> > > > > > > > > > > help a lot in the review process,
> > otherwise
> > > > most
> > > > > > > >> discussions
> > > > > > > >> > > > we have
> > > > > > > >> > > > > > > > > will
> > > > > > > >> > > > > > > > > > > focus on high level and lose important
> > > > details as
> > > > > > we
> > > > > > > >> > > > discover them in
> > > > > > > >> > > > > > > > > the
> > > > > > > >> > > > > > > > > > > post-agreement phase.
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > Boyang
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin
> > McCabe <
> > > > > > > >> > > > cmccabe@apache.org>
> > > > > > > >> > > > > > > > > wrote:
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose
> > Armando
> > > > > > Garcia
> > > > > > > >> Sancio
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I
> > have a
> > > > few
> > > > > > > >> comments
> > > > > > > >> > > > and
> > > > > > > >> > > > > > > > > questions.
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > In the KIP's Motivation and Overview
> > you
> > > > > > > >> mentioned the
> > > > > > > >> > > > > > > > LeaderAndIsr
> > > > > > > >> > > > > > > > > > and
> > > > > > > >> > > > > > > > > > > > > UpdateMetadata RPC. For example,
> > "updates
> > > > > > which
> > > > > > > >> the
> > > > > > > >> > > > controller
> > > > > > > >> > > > > > > > > > pushes,
> > > > > > > >> > > > > > > > > > > > such
> > > > > > > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata
> > > > messages".
> > > > > > Is
> > > > > > > >> your
> > > > > > > >> > > > thinking
> > > > > > > >> > > > > > > > that
> > > > > > > >> > > > > > > > > > we
> > > > > > > >> > > > > > > > > > > > will
> > > > > > > >> > > > > > > > > > > > > use MetadataFetch as a replacement to
> > just
> > > > > > > >> > > > UpdateMetadata only
> > > > > > > >> > > > > > > > and
> > > > > > > >> > > > > > > > > > add
> > > > > > > >> > > > > > > > > > > > > topic configuration in this state?
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > Hi Jose,
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > Thanks for taking a look.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to
> > > > replace
> > > > > > both
> > > > > > > >> > > > > > > > > > LeaderAndIsrRequest
> > > > > > > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic
> > > > configurations
> > > > > > > >> would be
> > > > > > > >> > > > fetched
> > > > > > > >> > > > > > > > > along
> > > > > > > >> > > > > > > > > > > > with the other metadata.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > In the section "Broker Metadata
> > > > Management",
> > > > > > you
> > > > > > > >> mention
> > > > > > > >> > > > "Just
> > > > > > > >> > > > > > > > like
> > > > > > > >> > > > > > > > > > > with
> > > > > > > >> > > > > > > > > > > > a
> > > > > > > >> > > > > > > > > > > > > fetch request, the broker will track
> > the
> > > > > > offset
> > > > > > > >> of the
> > > > > > > >> > > > last
> > > > > > > >> > > > > > > > updates
> > > > > > > >> > > > > > > > > > it
> > > > > > > >> > > > > > > > > > > > > fetched". To keep the log consistent
> > Raft
> > > > > > > >> requires that
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > > followers
> > > > > > > >> > > > > > > > > > > > keep
> > > > > > > >> > > > > > > > > > > > > all of the log entries (term/epoch and
> > > > offset)
> > > > > > > >> that are
> > > > > > > >> > > > after the
> > > > > > > >> > > > > > > > > > > > > highwatermark. Any log entry before
> > the
> > > > > > > >> highwatermark
> > > > > > > >> > > > can be
> > > > > > > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > > > > > > >> MetadataFetch API
> > > > > > > >> > > > to only
> > > > > > > >> > > > > > > > > return
> > > > > > > >> > > > > > > > > > > log
> > > > > > > >> > > > > > > > > > > > > entries up to the highwatermark?
> > Unlike
> > > > the
> > > > > > Raft
> > > > > > > >> > > > replication API
> > > > > > > >> > > > > > > > > > which
> > > > > > > >> > > > > > > > > > > > > will replicate/fetch log entries
> > after the
> > > > > > > >> highwatermark
> > > > > > > >> > > > for
> > > > > > > >> > > > > > > > > > consensus?
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't
> > expose
> > > > > > > >> metadata
> > > > > > > >> > > > updates to
> > > > > > > >> > > > > > > > the
> > > > > > > >> > > > > > > > > > > > brokers until they've been stored on a
> > > > majority
> > > > > > of
> > > > > > > >> the
> > > > > > > >> > > > Raft nodes.
> > > > > > > >> > > > > > > > > The
> > > > > > > >> > > > > > > > > > > > most obvious way to do that, like you
> > > > > > mentioned, is
> > > > > > > >> to
> > > > > > > >> > > > have the
> > > > > > > >> > > > > > > > > brokers
> > > > > > > >> > > > > > > > > > > > only fetch up to the HWM, but not
> > beyond.
> > > > There
> > > > > > > >> might be
> > > > > > > >> > > > a more
> > > > > > > >> > > > > > > > > clever
> > > > > > > >> > > > > > > > > > > way
> > > > > > > >> > > > > > > > > > > > to do it by fetching the data, but not
> > > > having
> > > > > > the
> > > > > > > >> brokers
> > > > > > > >> > > > act on it
> > > > > > > >> > > > > > > > > > until
> > > > > > > >> > > > > > > > > > > > the HWM advances.  I'm not sure if
> > that's
> > > > worth
> > > > > > it
> > > > > > > >> or
> > > > > > > >> > > > not.  We'll
> > > > > > > >> > > > > > > > > > discuss
> > > > > > > >> > > > > > > > > > > > this more in a separate KIP that just
> > > > discusses
> > > > > > > >> just Raft.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > In section "Broker Metadata
> > Management",
> > > > you
> > > > > > > >> mention "the
> > > > > > > >> > > > > > > > > controller
> > > > > > > >> > > > > > > > > > > will
> > > > > > > >> > > > > > > > > > > > > send a full metadata image rather
> > than a
> > > > > > series of
> > > > > > > >> > > > deltas". This
> > > > > > > >> > > > > > > > > KIP
> > > > > > > >> > > > > > > > > > > > > doesn't go into the set of operations
> > that
> > > > > > need
> > > > > > > >> to be
> > > > > > > >> > > > supported
> > > > > > > >> > > > > > > > on
> > > > > > > >> > > > > > > > > > top
> > > > > > > >> > > > > > > > > > > of
> > > > > > > >> > > > > > > > > > > > > Raft but it would be interested if
> > this
> > > > "full
> > > > > > > >> metadata
> > > > > > > >> > > > image"
> > > > > > > >> > > > > > > > could
> > > > > > > >> > > > > > > > > > be
> > > > > > > >> > > > > > > > > > > > > express also as deltas. For example,
> > > > assuming
> > > > > > we
> > > > > > > >> are
> > > > > > > >> > > > replicating
> > > > > > > >> > > > > > > > a
> > > > > > > >> > > > > > > > > > map
> > > > > > > >> > > > > > > > > > > > this
> > > > > > > >> > > > > > > > > > > > > "full metadata image" could be a
> > sequence
> > > > of
> > > > > > "put"
> > > > > > > >> > > > operations
> > > > > > > >> > > > > > > > > (znode
> > > > > > > >> > > > > > > > > > > > create
> > > > > > > >> > > > > > > > > > > > > to borrow ZK semantics).
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > The full image can definitely be
> > expressed
> > > > as a
> > > > > > sum
> > > > > > > >> of
> > > > > > > >> > > > deltas.  At
> > > > > > > >> > > > > > > > > some
> > > > > > > >> > > > > > > > > > > > point, the number of deltas will get
> > large
> > > > > > enough
> > > > > > > >> that
> > > > > > > >> > > > sending a
> > > > > > > >> > > > > > > > full
> > > > > > > >> > > > > > > > > > > image
> > > > > > > >> > > > > > > > > > > > is better, though.  One question that
> > we're
> > > > > > still
> > > > > > > >> thinking
> > > > > > > >> > > > about is
> > > > > > > >> > > > > > > > > how
> > > > > > > >> > > > > > > > > > > > much of this can be shared with generic
> > > > Kafka
> > > > > > log
> > > > > > > >> code,
> > > > > > > >> > > > and how
> > > > > > > >> > > > > > > > much
> > > > > > > >> > > > > > > > > > > should
> > > > > > > >> > > > > > > > > > > > be different.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > In section "Broker Metadata
> > Management",
> > > > you
> > > > > > > >> mention
> > > > > > > >> > > > "This
> > > > > > > >> > > > > > > > request
> > > > > > > >> > > > > > > > > > will
> > > > > > > >> > > > > > > > > > > > > double as a heartbeat, letting the
> > > > controller
> > > > > > > >> know that
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > broker
> > > > > > > >> > > > > > > > > is
> > > > > > > >> > > > > > > > > > > > > alive". In section "Broker State
> > > > Machine", you
> > > > > > > >> mention
> > > > > > > >> > > > "The
> > > > > > > >> > > > > > > > > > > MetadataFetch
> > > > > > > >> > > > > > > > > > > > > API serves as this registration
> > > > mechanism".
> > > > > > Does
> > > > > > > >> this
> > > > > > > >> > > > mean that
> > > > > > > >> > > > > > > > the
> > > > > > > >> > > > > > > > > > > > > MetadataFetch Request will optionally
> > > > include
> > > > > > > >> broker
> > > > > > > >> > > > > > > > configuration
> > > > > > > >> > > > > > > > > > > > > information?
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > I was originally thinking that the
> > > > > > > >> MetadataFetchRequest
> > > > > > > >> > > > should
> > > > > > > >> > > > > > > > > include
> > > > > > > >> > > > > > > > > > > > broker configuration information.
> > Thinking
> > > > > > about
> > > > > > > >> this
> > > > > > > >> > > > more, maybe
> > > > > > > >> > > > > > > > we
> > > > > > > >> > > > > > > > > > > > should just have a special registration
> > RPC
> > > > that
> > > > > > > >> contains
> > > > > > > >> > > > that
> > > > > > > >> > > > > > > > > > > information,
> > > > > > > >> > > > > > > > > > > > to avoid sending it over the wire all
> > the
> > > > time.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > Does this also mean that MetadataFetch
> > > > request
> > > > > > > >> will
> > > > > > > >> > > > result in
> > > > > > > >> > > > > > > > > > > > > a "write"/AppendEntries through the
> > Raft
> > > > > > > >> replication
> > > > > > > >> > > > protocol
> > > > > > > >> > > > > > > > > before
> > > > > > > >> > > > > > > > > > > you
> > > > > > > >> > > > > > > > > > > > > can send the associated MetadataFetch
> > > > > > Response?
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > I think we should require the broker to
> > be
> > > > out
> > > > > > of
> > > > > > > >> the
> > > > > > > >> > > > Offline state
> > > > > > > >> > > > > > > > > > > before
> > > > > > > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So
> > the
> > > > > > separate
> > > > > > > >> > > > registration
> > > > > > > >> > > > > > > > RPC
> > > > > > > >> > > > > > > > > > > > should have completed first.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > In section "Broker State", you mention
> > > > that a
> > > > > > > >> broker can
> > > > > > > >> > > > > > > > transition
> > > > > > > >> > > > > > > > > > to
> > > > > > > >> > > > > > > > > > > > > online after it is caught with the
> > > > metadata.
> > > > > > What
> > > > > > > >> do you
> > > > > > > >> > > > mean by
> > > > > > > >> > > > > > > > > > this?
> > > > > > > >> > > > > > > > > > > > > Metadata is always changing. How does
> > the
> > > > > > broker
> > > > > > > >> know
> > > > > > > >> > > > that it is
> > > > > > > >> > > > > > > > > > caught
> > > > > > > >> > > > > > > > > > > > up
> > > > > > > >> > > > > > > > > > > > > since it doesn't participate in the
> > > > consensus
> > > > > > or
> > > > > > > >> the
> > > > > > > >> > > > advancement
> > > > > > > >> > > > > > > > of
> > > > > > > >> > > > > > > > > > the
> > > > > > > >> > > > > > > > > > > > > highwatermark?
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > That's a good point.  Being "caught up"
> > is
> > > > > > somewhat
> > > > > > > >> of a
> > > > > > > >> > > > fuzzy
> > > > > > > >> > > > > > > > > concept
> > > > > > > >> > > > > > > > > > > > here, since the brokers do not
> > participate
> > > > in
> > > > > > the
> > > > > > > >> metadata
> > > > > > > >> > > > > > > > consensus.
> > > > > > > >> > > > > > > > > > I
> > > > > > > >> > > > > > > > > > > > think ideally we would want to define
> > it in
> > > > > > terms
> > > > > > > >> of time
> > > > > > > >> > > > ("the
> > > > > > > >> > > > > > > > > broker
> > > > > > > >> > > > > > > > > > > has
> > > > > > > >> > > > > > > > > > > > all the updates from the last 2
> > minutes",
> > > > for
> > > > > > > >> example.)
> > > > > > > >> > > > We should
> > > > > > > >> > > > > > > > > > spell
> > > > > > > >> > > > > > > > > > > > this out better in the KIP.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > In section "Start the controller
> > quorum
> > > > > > nodes",
> > > > > > > >> you
> > > > > > > >> > > > mention "Once
> > > > > > > >> > > > > > > > > it
> > > > > > > >> > > > > > > > > > > has
> > > > > > > >> > > > > > > > > > > > > taken over the /controller node, the
> > > > active
> > > > > > > >> controller
> > > > > > > >> > > > will
> > > > > > > >> > > > > > > > proceed
> > > > > > > >> > > > > > > > > > to
> > > > > > > >> > > > > > > > > > > > load
> > > > > > > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will
> > > > write
> > > > > > out
> > > > > > > >> this
> > > > > > > >> > > > information
> > > > > > > >> > > > > > > > to
> > > > > > > >> > > > > > > > > > the
> > > > > > > >> > > > > > > > > > > > > quorum's metadata storage.  After this
> > > > point,
> > > > > > the
> > > > > > > >> > > > metadata quorum
> > > > > > > >> > > > > > > > > > will
> > > > > > > >> > > > > > > > > > > be
> > > > > > > >> > > > > > > > > > > > > the metadata store of record, rather
> > than
> > > > the
> > > > > > > >> data in
> > > > > > > >> > > > ZooKeeper."
> > > > > > > >> > > > > > > > > > > During
> > > > > > > >> > > > > > > > > > > > > this migration do should we expect to
> > > > have a
> > > > > > > >> small period
> > > > > > > >> > > > > > > > > controller
> > > > > > > >> > > > > > > > > > > > > unavailability while the controller
> > > > replicas
> > > > > > this
> > > > > > > >> state
> > > > > > > >> > > > to all of
> > > > > > > >> > > > > > > > > the
> > > > > > > >> > > > > > > > > > > > raft
> > > > > > > >> > > > > > > > > > > > > nodes in the controller quorum and we
> > > > buffer
> > > > > > new
> > > > > > > >> > > > controller API
> > > > > > > >> > > > > > > > > > > requests?
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > Yes, the controller would be unavailable
> > > > during
> > > > > > this
> > > > > > > >> > > > time.  I don't
> > > > > > > >> > > > > > > > > > think
> > > > > > > >> > > > > > > > > > > > this will be that different from the
> > current
> > > > > > period
> > > > > > > >> of
> > > > > > > >> > > > > > > > unavailability
> > > > > > > >> > > > > > > > > > > when
> > > > > > > >> > > > > > > > > > > > a new controller starts up and needs to
> > > > load the
> > > > > > > >> full
> > > > > > > >> > > > state from
> > > > > > > >> > > > > > > > ZK.
> > > > > > > >> > > > > > > > > > The
> > > > > > > >> > > > > > > > > > > > main difference is that in this period,
> > we'd
> > > > > > have
> > > > > > > >> to write
> > > > > > > >> > > > to the
> > > > > > > >> > > > > > > > > > > > controller quorum rather than just to
> > > > memory.
> > > > > > But
> > > > > > > >> we
> > > > > > > >> > > > believe this
> > > > > > > >> > > > > > > > > > should
> > > > > > > >> > > > > > > > > > > > be pretty fast.
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > regards,
> > > > > > > >> > > > > > > > > > > > Colin
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > Thanks!
> > > > > > > >> > > > > > > > > > > > > -Jose
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > > --
> > > > > > > >> > > David Arthur
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ryanne Dolan <ry...@gmail.com>.
Thanks Colin, sgtm. Please make this clear in the KIP -- otherwise it is
hard to nail down what we are voting for.

Ryanne


On Fri, Aug 23, 2019, 12:58 PM Colin McCabe <cm...@apache.org> wrote:

> On Fri, Aug 23, 2019, at 06:24, Ryanne Dolan wrote:
> > Colin, can you outline what specifically would be in scope for this KIP
> vs
> > deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
> > section? Is the idea to get to the bridge release with this KIP, and then
> > go from there?
> >
> > Ryanne
> >
>
> Hi Ryanne,
>
> The goal for KIP-500 is to set out an overall vision for how we will
> remove ZooKeeper and transition to managing metadata via a controller
> quorum.
>
> We will create follow-on KIPs that will lay out the specific details of
> each step.
>
> * A KIP for allowing kafka-configs.sh to change topic configurations
> without using ZooKeeper.  (It can already change broker configurations
> without ZK)
>
> * A KIP for adding APIs to replace direct ZK access by the brokers.
>
> * A KIP to describe Raft replication in Kafka, including the overall
> protocol, details of each RPC, etc.
>
> * A KIP describing the controller changes, how metadata is stored, etc.
>
> There may be other KIPs that we need (for example, if we find another tool
> that still has a hard ZK dependency), but that's the general idea.  KIP-500
> is about the overall design-- the follow on KIPs are about the specific
> details.
>
> best,
> Colin
>
>
> >
> > On Thu, Aug 22, 2019, 11:58 AM Colin McCabe <cm...@apache.org> wrote:
> >
> > > On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> > > > Thanks, Colin.  The changes you made to the KIP related to the bridge
> > > > release help make it clearer.  I still have some confusion about the
> > > phrase
> > > > "The rolling upgrade from the bridge release will take several
> steps."
> > > > This made me think you are talking about moving from the bridge
> release
> > > to
> > > > some other, newer, release that comes after the bridge release.  But
> I
> > > > think what you are getting at is that the bridge release can be run
> with
> > > or
> > > > without Zookeeper -- when first upgrading to it Zookeeper remains in
> use,
> > > > but then there is a transition that can be made to engage the warp
> > > drive...
> > > > I mean the Controller Quorum.  So maybe the phrase should be "The
> rolling
> > > > upgrade through the bridge release -- starting with Zookeeper being
> in
> > > use
> > > > and ending with Zookeeper having been replaced by the Controller
> Quorum
> > > --
> > > > will take several steps."
> > >
> > > Hi Ron,
> > >
> > > To clarify, the bridge release will require ZooKeeper.  It will also
> not
> > > support the controller quorum.  It's a bridge in the sense that you
> must
> > > upgrade to a bridge release prior to upgrading to a ZK-less release.  I
> > > added some more descriptive text to the bridge release paragraph--
> > > hopefully this makes it clearer.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Do I understand it correctly, and might some change in phrasing or
> > > > additional clarification help others avoid the same confusion I had?
> > > >
> > > > Ron
> > > >
> > > > On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org>
> wrote:
> > > >
> > > > > On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> > > > > > Hi Colin.  I like the concept of a "bridge release" for migrating
> > > off of
> > > > > > Zookeeper, but I worry that it may become a bottleneck if people
> > > hesitate
> > > > > > to replace Zookeeper -- they would be unable to adopt newer
> versions
> > > of
> > > > > > Kafka until taking (what feels to them like) a giant leap.  As an
> > > > > example,
> > > > > > assuming version 4.0.x of Kafka is the supported bridge release,
> I
> > > would
> > > > > > not be surprised if uptake of the 4.x release and the time-based
> > > releases
> > > > > > that follow it end up being much slower due to the perceived
> barrier.
> > > > > >
> > > > > > Any perceived barrier could be lowered if the 4.0.x release could
> > > > > > optionally continue to use Zookeeper -- then the cutover would
> be two
> > > > > > incremental steps (move to 4.0.x, then replace Zookeeper while
> > > staying on
> > > > > > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and
> replace
> > > > > > Zookeeper in one fell swoop).
> > > > >
> > > > > Hi Ron,
> > > > >
> > > > > Just to clarify, the "bridge release" will continue to use
> ZooKeeper.
> > > It
> > > > > will not support running without ZooKeeper.  It is the releases
> that
> > > follow
> > > > > the bridge release that will remove ZooKeeper.
> > > > >
> > > > > Also, it's a bit unclear whether the bridge release would be 3.x or
> > > 4.x,
> > > > > or something to follow.  We do know that the bridge release can't
> be a
> > > 2.x
> > > > > release, since it requires at least one incompatible change,
> removing
> > > > > --zookeeper options from all the shell scripts.  (Since we're doing
> > > > > semantic versioning, any time we make an incompatible change, we
> bump
> > > the
> > > > > major version number.)
> > > > >
> > > > > In general, using two sources of metadata is a lot more complex and
> > > > > error-prone than one.  A lot of the bugs and corner cases we have
> are
> > > the
> > > > > result of divergences between the controller and the state in
> > > ZooKeeper.
> > > > > Eliminating this divergence, and the split-brain scenarios it
> creates,
> > > is a
> > > > > major goal of this work.
> > > > >
> > > > > >
> > > > > > Regardless of whether what I wrote above has merit or not, I
> think
> > > the
> > > > > KIP
> > > > > > should be more explicit about what the upgrade constraints
> actually
> > > are.
> > > > > > Can the bridge release be adopted with Zookeeper remaining in
> place
> > > and
> > > > > > then cutting over as a second, follow-on step, or must the
> Controller
> > > > > > Quorum nodes be started first and the bridge release cannot be
> used
> > > with
> > > > > > Zookeeper at all?
> > > > >
> > > > > As I mentioned above, the bridge release supports (indeed,
> requires)
> > > > > ZooKeeper.  I have added a little more text about this to KIP-500
> which
> > > > > hopefully makes it clearer.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > >  If the bridge release cannot be used with Zookeeper at
> > > > > > all, then no version at or beyond the bridge release is available
> > > > > > unless/until abandoning Zookeeper; if the bridge release can be
> used
> > > with
> > > > > > Zookeeper, then is it the only version that can be used with
> > > Zookeeper,
> > > > > or
> > > > > > can Zookeeper be kept for additional releases if desired?
> > > > > >
> > > > > > Ron
> > > > > >
> > > > > > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <
> rndgstn@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Hi Colin.  The diagram up at the top confused me --
> specifically,
> > > the
> > > > > > > lines connecting the controller/active-controller to the
> brokers.
> > > I
> > > > > had
> > > > > > > assumed the arrows on those lines represented the direction of
> data
> > > > > flow,
> > > > > > > but that is not the case; the arrows actually identify the
> target
> > > of
> > > > > the
> > > > > > > action, and the non-arrowed end indicates the initiator of the
> > > > > action.  For
> > > > > > > example, the lines point from the controller to the brokers in
> the
> > > > > "today"
> > > > > > > section on the left to show that the controller pushes to the
> > > brokers;
> > > > > the
> > > > > > > lines point from the brokers to the active-controller in the
> > > "tomorrow"
> > > > > > > section on the right to show that the brokers pull from the
> > > > > > > active-controller.  As I said, this confused me because my gut
> > > > > instinct was
> > > > > > > to interpret the arrow as indicating the direction of data
> flow,
> > > and
> > > > > when I
> > > > > > > look at the "tomorrow" picture on the right I initially thought
> > > > > information
> > > > > > > was moving from the brokers to the active-controller.  Did you
> > > consider
> > > > > > > drawing that picture with the arrows reversed in the "tomorrow"
> > > side so
> > > > > > > that the arrows represent the direction of data flow, and then
> add
> > > the
> > > > > > > labels "push" on the "today" side and "pull" on the "tomorrow"
> > > side to
> > > > > > > indicate who initiates the data flow?  It occurs to me that
> this
> > > > > picture
> > > > > > > may end up being widely distributed, so it might be in
> everyone's
> > > > > interest
> > > > > > > to proactively avoid any possible confusion by being more
> explicit.
> > > > > > >
> > > > > > > Minor corrections?
> > > > > > > <<<In the current world, a broker which can contact ZooKeeper
> but
> > > which
> > > > > > > is partitioned from the active controller
> > > > > > > >>>In the current world, a broker which can contact ZooKeeper
> but
> > > which
> > > > > > > is partitioned from the controller
> > > > > > >
> > > > > > > <<<Eventually, the controller will ask the broker to finally go
> > > offline
> > > > > > > >>>Eventually, the active controller will ask the broker to
> > > finally go
> > > > > > > offline
> > > > > > >
> > > > > > > <<<New versions of the clients should send these operations
> > > directly to
> > > > > > > the controller
> > > > > > > >>>New versions of the clients should send these operations
> > > directly to
> > > > > > > the active controller
> > > > > > >
> > > > > > > <<<In the post-ZK world, the leader will make an RPC to the
> > > controller
> > > > > > > instead
> > > > > > > >>>In the post-ZK world, the leader will make an RPC to the
> active
> > > > > > > controller instead
> > > > > > >
> > > > > > > <<<For example, the brokers may need to forward their requests
> to
> > > the
> > > > > > > controller.
> > > > > > > >>>For example, the brokers may need to forward their requests
> to
> > > the
> > > > > > > active controller.
> > > > > > >
> > > > > > > <<<The new controller will monitor ZooKeeper for legacy broker
> node
> > > > > > > registrations
> > > > > > > >>>The new (active) controller will monitor ZooKeeper for
> legacy
> > > broker
> > > > > > > node registrations
> > > > > > >
> > > > > > > Ron
> > > > > > >
> > > > > > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <
> cmccabe@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > >> Hi all,
> > > > > > >>
> > > > > > >> The KIP has been out for a while, so I'm thinking about
> calling a
> > > vote
> > > > > > >> some time this week.
> > > > > > >>
> > > > > > >> best,
> > > > > > >> Colin
> > > > > > >>
> > > > > > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > > > > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > > > > >> > > Thanks for the KIP, Colin. This looks great!
> > > > > > >> > >
> > > > > > >> > > I really like the idea of separating the Controller and
> Broker
> > > > > JVMs.
> > > > > > >> > >
> > > > > > >> > > As you alluded to above, it might be nice to have a
> separate
> > > > > > >> > > broker-registration API to avoid overloading the metadata
> > > fetch
> > > > > API.
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Hi David,
> > > > > > >> >
> > > > > > >> > Thanks for taking a look.
> > > > > > >> >
> > > > > > >> > I removed the sentence about MetadataFetch also serving as
> the
> > > > > broker
> > > > > > >> > registration API.  I think I agree that we will probably
> want a
> > > > > > >> > separate RPC to fill this role.  We will have a follow-on
> KIP
> > > that
> > > > > will
> > > > > > >> > go into more detail about metadata propagation and
> registration
> > > in
> > > > > the
> > > > > > >> > post-ZK world.  That KIP will also have a full description
> of
> > > the
> > > > > > >> > registration RPC, etc.  For now, I think the important part
> for
> > > > > KIP-500
> > > > > > >> > is that the broker registers with the controller quorum.  On
> > > > > > >> > registration, the controller quorum assigns it a new broker
> > > epoch,
> > > > > > >> > which can distinguish successive broker incarnations.
> > > > > > >> >
> > > > > > >> > >
> > > > > > >> > > When a broker gets a metadata delta, will it be a
> sequence of
> > > > > deltas
> > > > > > >> since
> > > > > > >> > > the last update or a cumulative delta since the last
> update?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > It will be a sequence of deltas.  Basically, the broker
> will be
> > > > > reading
> > > > > > >> > from the metadata log.
> > > > > > >> >
> > > > > > >> > >
> > > > > > >> > > Will we include any kind of integrity check on the deltas
> to
> > > > > ensure
> > > > > > >> the brokers
> > > > > > >> > > have applied them correctly? Perhaps this will be
> addressed in
> > > > > one of
> > > > > > >> the
> > > > > > >> > > follow-on KIPs.
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > In general, we will have checksums on the metadata that we
> > > fetch.
> > > > > This
> > > > > > >> > is similar to how we have checksums on regular data.  Or if
> the
> > > > > > >> > question is about catching logic errors in the metadata
> handling
> > > > > code,
> > > > > > >> > that sounds more like something that should be caught by
> test
> > > cases.
> > > > > > >> >
> > > > > > >> > best,
> > > > > > >> > Colin
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > Thanks!
> > > > > > >> > >
> > > > > > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <
> > > cmccabe@apache.org>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Hi Mickael,
> > > > > > >> > > >
> > > > > > >> > > > Thanks for taking a look.
> > > > > > >> > > >
> > > > > > >> > > > I don't think we want to support that kind of
> multi-tenancy
> > > at
> > > > > the
> > > > > > >> > > > controller level.  If the cluster is small enough that
> we
> > > want
> > > > > to
> > > > > > >> pack the
> > > > > > >> > > > controller(s) with something else, we could run them
> > > alongside
> > > > > the
> > > > > > >> brokers,
> > > > > > >> > > > or possibly inside three of the broker JVMs.
> > > > > > >> > > >
> > > > > > >> > > > best,
> > > > > > >> > > > Colin
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > > > >> > > > > Thank Colin for kickstarting this initiative.
> > > > > > >> > > > >
> > > > > > >> > > > > Just one question.
> > > > > > >> > > > > - A nice feature of Zookeeper is the ability to use
> > > chroots
> > > > > and
> > > > > > >> have
> > > > > > >> > > > > several Kafka clusters use the same Zookeeper
> ensemble. Is
> > > > > this
> > > > > > >> > > > > something we should keep?
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks
> > > > > > >> > > > >
> > > > > > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> > > > > cmccabe@apache.org>
> > > > > > >> wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > > > >> > > > > > > Hi Colin,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Thanks for the KIP.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Currently ZooKeeper provides a convenient
> notification
> > > > > > >> mechanism for
> > > > > > >> > > > > > > knowing that broker and topic configuration has
> > > changed.
> > > > > While
> > > > > > >> > > > KIP-500 does
> > > > > > >> > > > > > > suggest that incremental metadata update is
> expected
> > > to
> > > > > come
> > > > > > >> to
> > > > > > >> > > > clients
> > > > > > >> > > > > > > eventually, that would seem to imply that for some
> > > number
> > > > > of
> > > > > > >> > > > releases there
> > > > > > >> > > > > > > would be no equivalent mechanism for knowing about
> > > config
> > > > > > >> changes.
> > > > > > >> > > > Is there
> > > > > > >> > > > > > > any thinking at this point about how a similar
> > > > > notification
> > > > > > >> might be
> > > > > > >> > > > > > > provided in the future?
> > > > > > >> > > > > >
> > > > > > >> > > > > > We could eventually have some inotify-like mechanism
> > > where
> > > > > > >> clients
> > > > > > >> > > > could register interest in various types of events and
> got
> > > > > notified
> > > > > > >> when
> > > > > > >> > > > they happened.  Reading the metadata log is conceptually
> > > simple.
> > > > > > >> The main
> > > > > > >> > > > complexity would be in setting up an API that made
> sense and
> > > > > that
> > > > > > >> didn't
> > > > > > >> > > > unduly constrain future implementations.  We'd have to
> think
> > > > > > >> carefully
> > > > > > >> > > > about what the real use-cases for this were, though.
> > > > > > >> > > > > >
> > > > > > >> > > > > > best,
> > > > > > >> > > > > > Colin
> > > > > > >> > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Thanks,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Tom
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor
> Somogyi-Vass <
> > > > > > >> > > > viktorsomogyi@gmail.com>
> > > > > > >> > > > > > > wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > Hey Colin,
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > I think this is a long-awaited KIP, thanks for
> > > driving
> > > > > it.
> > > > > > >> I'm
> > > > > > >> > > > excited to
> > > > > > >> > > > > > > > see this in Kafka once. I collected my questions
> > > (and I
> > > > > > >> accept the
> > > > > > >> > > > "TBD"
> > > > > > >> > > > > > > > answer as they might be a bit deep for this high
> > > level
> > > > > :) ).
> > > > > > >> > > > > > > > 1.) Are there any specific reasons for the
> > > Controller
> > > > > just
> > > > > > >> > > > periodically
> > > > > > >> > > > > > > > persisting its state on disk periodically
> instead of
> > > > > > >> > > > asynchronously with
> > > > > > >> > > > > > > > every update? Wouldn't less frequent saves
> increase
> > > the
> > > > > > >> chance for
> > > > > > >> > > > missing
> > > > > > >> > > > > > > > a state change if the controller crashes
> between two
> > > > > saves?
> > > > > > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata
> > > from
> > > > > the
> > > > > > >> follower
> > > > > > >> > > > > > > > controllers? I assume that followers would have
> > > > > up-to-date
> > > > > > >> > > > information
> > > > > > >> > > > > > > > therefore brokers could fetch from there in
> theory.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Thanks,
> > > > > > >> > > > > > > > Viktor
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > > > > >> > > > reluctanthero104@gmail.com>
> > > > > > >> > > > > > > > wrote:
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > > Thanks for explaining Ismael! Breaking down
> into
> > > > > > >> follow-up KIPs
> > > > > > >> > > > sounds
> > > > > > >> > > > > > > > like
> > > > > > >> > > > > > > > > a good idea.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > > > > > >> ismael@juma.me.uk>
> > > > > > >> > > > wrote:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > > Hi Boyang,
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > > > Yes, there will be several KIPs that will
> > > discuss
> > > > > the
> > > > > > >> items you
> > > > > > >> > > > > > > > describe
> > > > > > >> > > > > > > > > in
> > > > > > >> > > > > > > > > > detail. Colin, it may be helpful to make
> this
> > > clear
> > > > > in
> > > > > > >> the KIP
> > > > > > >> > > > 500
> > > > > > >> > > > > > > > > > description.
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > > > Ismael
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > > > > >> > > > reluctanthero104@gmail.com
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > > wrote:
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > > > > Thanks Colin for initiating this important
> > > effort!
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > > > One question I have is whether we have a
> > > session
> > > > > > >> discussing
> > > > > > >> > > > the
> > > > > > >> > > > > > > > > > controller
> > > > > > >> > > > > > > > > > > failover in the new architecture? I know
> we
> > > are
> > > > > using
> > > > > > >> Raft
> > > > > > >> > > > protocol
> > > > > > >> > > > > > > > to
> > > > > > >> > > > > > > > > > > failover, yet it's still valuable to
> discuss
> > > the
> > > > > > >> steps new
> > > > > > >> > > > cluster is
> > > > > > >> > > > > > > > > > going
> > > > > > >> > > > > > > > > > > to take to reach the stable stage again,
> so
> > > that
> > > > > we
> > > > > > >> could
> > > > > > >> > > > easily
> > > > > > >> > > > > > > > > measure
> > > > > > >> > > > > > > > > > > the availability of the metadata servers.
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > > > Another suggestion I have is to write a
> > > > > step-by-step
> > > > > > >> design
> > > > > > >> > > > doc like
> > > > > > >> > > > > > > > > what
> > > > > > >> > > > > > > > > > > we did in KIP-98
> > > > > > >> > > > > > > > > > > <
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > >
> > > > > > >>
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > >> > > > > > > > > > > >,
> > > > > > >> > > > > > > > > > > including the new request protocols and
> how
> > > they
> > > > > are
> > > > > > >> > > > interacting in
> > > > > > >> > > > > > > > the
> > > > > > >> > > > > > > > > > new
> > > > > > >> > > > > > > > > > > cluster. For a complicated change like
> this,
> > > an
> > > > > > >> > > > implementation design
> > > > > > >> > > > > > > > > doc
> > > > > > >> > > > > > > > > > > help a lot in the review process,
> otherwise
> > > most
> > > > > > >> discussions
> > > > > > >> > > > we have
> > > > > > >> > > > > > > > > will
> > > > > > >> > > > > > > > > > > focus on high level and lose important
> > > details as
> > > > > we
> > > > > > >> > > > discover them in
> > > > > > >> > > > > > > > > the
> > > > > > >> > > > > > > > > > > post-agreement phase.
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > > > Boyang
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin
> McCabe <
> > > > > > >> > > > cmccabe@apache.org>
> > > > > > >> > > > > > > > > wrote:
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose
> Armando
> > > > > Garcia
> > > > > > >> Sancio
> > > > > > >> > > > wrote:
> > > > > > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I
> have a
> > > few
> > > > > > >> comments
> > > > > > >> > > > and
> > > > > > >> > > > > > > > > questions.
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > In the KIP's Motivation and Overview
> you
> > > > > > >> mentioned the
> > > > > > >> > > > > > > > LeaderAndIsr
> > > > > > >> > > > > > > > > > and
> > > > > > >> > > > > > > > > > > > > UpdateMetadata RPC. For example,
> "updates
> > > > > which
> > > > > > >> the
> > > > > > >> > > > controller
> > > > > > >> > > > > > > > > > pushes,
> > > > > > >> > > > > > > > > > > > such
> > > > > > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata
> > > messages".
> > > > > Is
> > > > > > >> your
> > > > > > >> > > > thinking
> > > > > > >> > > > > > > > that
> > > > > > >> > > > > > > > > > we
> > > > > > >> > > > > > > > > > > > will
> > > > > > >> > > > > > > > > > > > > use MetadataFetch as a replacement to
> just
> > > > > > >> > > > UpdateMetadata only
> > > > > > >> > > > > > > > and
> > > > > > >> > > > > > > > > > add
> > > > > > >> > > > > > > > > > > > > topic configuration in this state?
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > Hi Jose,
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > Thanks for taking a look.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to
> > > replace
> > > > > both
> > > > > > >> > > > > > > > > > LeaderAndIsrRequest
> > > > > > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic
> > > configurations
> > > > > > >> would be
> > > > > > >> > > > fetched
> > > > > > >> > > > > > > > > along
> > > > > > >> > > > > > > > > > > > with the other metadata.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > In the section "Broker Metadata
> > > Management",
> > > > > you
> > > > > > >> mention
> > > > > > >> > > > "Just
> > > > > > >> > > > > > > > like
> > > > > > >> > > > > > > > > > > with
> > > > > > >> > > > > > > > > > > > a
> > > > > > >> > > > > > > > > > > > > fetch request, the broker will track
> the
> > > > > offset
> > > > > > >> of the
> > > > > > >> > > > last
> > > > > > >> > > > > > > > updates
> > > > > > >> > > > > > > > > > it
> > > > > > >> > > > > > > > > > > > > fetched". To keep the log consistent
> Raft
> > > > > > >> requires that
> > > > > > >> > > > the
> > > > > > >> > > > > > > > > followers
> > > > > > >> > > > > > > > > > > > keep
> > > > > > >> > > > > > > > > > > > > all of the log entries (term/epoch and
> > > offset)
> > > > > > >> that are
> > > > > > >> > > > after the
> > > > > > >> > > > > > > > > > > > > highwatermark. Any log entry before
> the
> > > > > > >> highwatermark
> > > > > > >> > > > can be
> > > > > > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > > > > > >> MetadataFetch API
> > > > > > >> > > > to only
> > > > > > >> > > > > > > > > return
> > > > > > >> > > > > > > > > > > log
> > > > > > >> > > > > > > > > > > > > entries up to the highwatermark?
> Unlike
> > > the
> > > > > Raft
> > > > > > >> > > > replication API
> > > > > > >> > > > > > > > > > which
> > > > > > >> > > > > > > > > > > > > will replicate/fetch log entries
> after the
> > > > > > >> highwatermark
> > > > > > >> > > > for
> > > > > > >> > > > > > > > > > consensus?
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't
> expose
> > > > > > >> metadata
> > > > > > >> > > > updates to
> > > > > > >> > > > > > > > the
> > > > > > >> > > > > > > > > > > > brokers until they've been stored on a
> > > majority
> > > > > of
> > > > > > >> the
> > > > > > >> > > > Raft nodes.
> > > > > > >> > > > > > > > > The
> > > > > > >> > > > > > > > > > > > most obvious way to do that, like you
> > > > > mentioned, is
> > > > > > >> to
> > > > > > >> > > > have the
> > > > > > >> > > > > > > > > brokers
> > > > > > >> > > > > > > > > > > > only fetch up to the HWM, but not
> beyond.
> > > There
> > > > > > >> might be
> > > > > > >> > > > a more
> > > > > > >> > > > > > > > > clever
> > > > > > >> > > > > > > > > > > way
> > > > > > >> > > > > > > > > > > > to do it by fetching the data, but not
> > > having
> > > > > the
> > > > > > >> brokers
> > > > > > >> > > > act on it
> > > > > > >> > > > > > > > > > until
> > > > > > >> > > > > > > > > > > > the HWM advances.  I'm not sure if
> that's
> > > worth
> > > > > it
> > > > > > >> or
> > > > > > >> > > > not.  We'll
> > > > > > >> > > > > > > > > > discuss
> > > > > > >> > > > > > > > > > > > this more in a separate KIP that just
> > > discusses
> > > > > > >> just Raft.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > In section "Broker Metadata
> Management",
> > > you
> > > > > > >> mention "the
> > > > > > >> > > > > > > > > controller
> > > > > > >> > > > > > > > > > > will
> > > > > > >> > > > > > > > > > > > > send a full metadata image rather
> than a
> > > > > series of
> > > > > > >> > > > deltas". This
> > > > > > >> > > > > > > > > KIP
> > > > > > >> > > > > > > > > > > > > doesn't go into the set of operations
> that
> > > > > need
> > > > > > >> to be
> > > > > > >> > > > supported
> > > > > > >> > > > > > > > on
> > > > > > >> > > > > > > > > > top
> > > > > > >> > > > > > > > > > > of
> > > > > > >> > > > > > > > > > > > > Raft but it would be interested if
> this
> > > "full
> > > > > > >> metadata
> > > > > > >> > > > image"
> > > > > > >> > > > > > > > could
> > > > > > >> > > > > > > > > > be
> > > > > > >> > > > > > > > > > > > > express also as deltas. For example,
> > > assuming
> > > > > we
> > > > > > >> are
> > > > > > >> > > > replicating
> > > > > > >> > > > > > > > a
> > > > > > >> > > > > > > > > > map
> > > > > > >> > > > > > > > > > > > this
> > > > > > >> > > > > > > > > > > > > "full metadata image" could be a
> sequence
> > > of
> > > > > "put"
> > > > > > >> > > > operations
> > > > > > >> > > > > > > > > (znode
> > > > > > >> > > > > > > > > > > > create
> > > > > > >> > > > > > > > > > > > > to borrow ZK semantics).
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > The full image can definitely be
> expressed
> > > as a
> > > > > sum
> > > > > > >> of
> > > > > > >> > > > deltas.  At
> > > > > > >> > > > > > > > > some
> > > > > > >> > > > > > > > > > > > point, the number of deltas will get
> large
> > > > > enough
> > > > > > >> that
> > > > > > >> > > > sending a
> > > > > > >> > > > > > > > full
> > > > > > >> > > > > > > > > > > image
> > > > > > >> > > > > > > > > > > > is better, though.  One question that
> we're
> > > > > still
> > > > > > >> thinking
> > > > > > >> > > > about is
> > > > > > >> > > > > > > > > how
> > > > > > >> > > > > > > > > > > > much of this can be shared with generic
> > > Kafka
> > > > > log
> > > > > > >> code,
> > > > > > >> > > > and how
> > > > > > >> > > > > > > > much
> > > > > > >> > > > > > > > > > > should
> > > > > > >> > > > > > > > > > > > be different.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > In section "Broker Metadata
> Management",
> > > you
> > > > > > >> mention
> > > > > > >> > > > "This
> > > > > > >> > > > > > > > request
> > > > > > >> > > > > > > > > > will
> > > > > > >> > > > > > > > > > > > > double as a heartbeat, letting the
> > > controller
> > > > > > >> know that
> > > > > > >> > > > the
> > > > > > >> > > > > > > > broker
> > > > > > >> > > > > > > > > is
> > > > > > >> > > > > > > > > > > > > alive". In section "Broker State
> > > Machine", you
> > > > > > >> mention
> > > > > > >> > > > "The
> > > > > > >> > > > > > > > > > > MetadataFetch
> > > > > > >> > > > > > > > > > > > > API serves as this registration
> > > mechanism".
> > > > > Does
> > > > > > >> this
> > > > > > >> > > > mean that
> > > > > > >> > > > > > > > the
> > > > > > >> > > > > > > > > > > > > MetadataFetch Request will optionally
> > > include
> > > > > > >> broker
> > > > > > >> > > > > > > > configuration
> > > > > > >> > > > > > > > > > > > > information?
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > I was originally thinking that the
> > > > > > >> MetadataFetchRequest
> > > > > > >> > > > should
> > > > > > >> > > > > > > > > include
> > > > > > >> > > > > > > > > > > > broker configuration information.
> Thinking
> > > > > about
> > > > > > >> this
> > > > > > >> > > > more, maybe
> > > > > > >> > > > > > > > we
> > > > > > >> > > > > > > > > > > > should just have a special registration
> RPC
> > > that
> > > > > > >> contains
> > > > > > >> > > > that
> > > > > > >> > > > > > > > > > > information,
> > > > > > >> > > > > > > > > > > > to avoid sending it over the wire all
> the
> > > time.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > Does this also mean that MetadataFetch
> > > request
> > > > > > >> will
> > > > > > >> > > > result in
> > > > > > >> > > > > > > > > > > > > a "write"/AppendEntries through the
> Raft
> > > > > > >> replication
> > > > > > >> > > > protocol
> > > > > > >> > > > > > > > > before
> > > > > > >> > > > > > > > > > > you
> > > > > > >> > > > > > > > > > > > > can send the associated MetadataFetch
> > > > > Response?
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > I think we should require the broker to
> be
> > > out
> > > > > of
> > > > > > >> the
> > > > > > >> > > > Offline state
> > > > > > >> > > > > > > > > > > before
> > > > > > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So
> the
> > > > > separate
> > > > > > >> > > > registration
> > > > > > >> > > > > > > > RPC
> > > > > > >> > > > > > > > > > > > should have completed first.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > In section "Broker State", you mention
> > > that a
> > > > > > >> broker can
> > > > > > >> > > > > > > > transition
> > > > > > >> > > > > > > > > > to
> > > > > > >> > > > > > > > > > > > > online after it is caught with the
> > > metadata.
> > > > > What
> > > > > > >> do you
> > > > > > >> > > > mean by
> > > > > > >> > > > > > > > > > this?
> > > > > > >> > > > > > > > > > > > > Metadata is always changing. How does
> the
> > > > > broker
> > > > > > >> know
> > > > > > >> > > > that it is
> > > > > > >> > > > > > > > > > caught
> > > > > > >> > > > > > > > > > > > up
> > > > > > >> > > > > > > > > > > > > since it doesn't participate in the
> > > consensus
> > > > > or
> > > > > > >> the
> > > > > > >> > > > advancement
> > > > > > >> > > > > > > > of
> > > > > > >> > > > > > > > > > the
> > > > > > >> > > > > > > > > > > > > highwatermark?
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > That's a good point.  Being "caught up"
> is
> > > > > somewhat
> > > > > > >> of a
> > > > > > >> > > > fuzzy
> > > > > > >> > > > > > > > > concept
> > > > > > >> > > > > > > > > > > > here, since the brokers do not
> participate
> > > in
> > > > > the
> > > > > > >> metadata
> > > > > > >> > > > > > > > consensus.
> > > > > > >> > > > > > > > > > I
> > > > > > >> > > > > > > > > > > > think ideally we would want to define
> it in
> > > > > terms
> > > > > > >> of time
> > > > > > >> > > > ("the
> > > > > > >> > > > > > > > > broker
> > > > > > >> > > > > > > > > > > has
> > > > > > >> > > > > > > > > > > > all the updates from the last 2
> minutes",
> > > for
> > > > > > >> example.)
> > > > > > >> > > > We should
> > > > > > >> > > > > > > > > > spell
> > > > > > >> > > > > > > > > > > > this out better in the KIP.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > In section "Start the controller
> quorum
> > > > > nodes",
> > > > > > >> you
> > > > > > >> > > > mention "Once
> > > > > > >> > > > > > > > > it
> > > > > > >> > > > > > > > > > > has
> > > > > > >> > > > > > > > > > > > > taken over the /controller node, the
> > > active
> > > > > > >> controller
> > > > > > >> > > > will
> > > > > > >> > > > > > > > proceed
> > > > > > >> > > > > > > > > > to
> > > > > > >> > > > > > > > > > > > load
> > > > > > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will
> > > write
> > > > > out
> > > > > > >> this
> > > > > > >> > > > information
> > > > > > >> > > > > > > > to
> > > > > > >> > > > > > > > > > the
> > > > > > >> > > > > > > > > > > > > quorum's metadata storage.  After this
> > > point,
> > > > > the
> > > > > > >> > > > metadata quorum
> > > > > > >> > > > > > > > > > will
> > > > > > >> > > > > > > > > > > be
> > > > > > >> > > > > > > > > > > > > the metadata store of record, rather
> than
> > > the
> > > > > > >> data in
> > > > > > >> > > > ZooKeeper."
> > > > > > >> > > > > > > > > > > During
> > > > > > >> > > > > > > > > > > > > this migration do should we expect to
> > > have a
> > > > > > >> small period
> > > > > > >> > > > > > > > > controller
> > > > > > >> > > > > > > > > > > > > unavailability while the controller
> > > replicas
> > > > > this
> > > > > > >> state
> > > > > > >> > > > to all of
> > > > > > >> > > > > > > > > the
> > > > > > >> > > > > > > > > > > > raft
> > > > > > >> > > > > > > > > > > > > nodes in the controller quorum and we
> > > buffer
> > > > > new
> > > > > > >> > > > controller API
> > > > > > >> > > > > > > > > > > requests?
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > Yes, the controller would be unavailable
> > > during
> > > > > this
> > > > > > >> > > > time.  I don't
> > > > > > >> > > > > > > > > > think
> > > > > > >> > > > > > > > > > > > this will be that different from the
> current
> > > > > period
> > > > > > >> of
> > > > > > >> > > > > > > > unavailability
> > > > > > >> > > > > > > > > > > when
> > > > > > >> > > > > > > > > > > > a new controller starts up and needs to
> > > load the
> > > > > > >> full
> > > > > > >> > > > state from
> > > > > > >> > > > > > > > ZK.
> > > > > > >> > > > > > > > > > The
> > > > > > >> > > > > > > > > > > > main difference is that in this period,
> we'd
> > > > > have
> > > > > > >> to write
> > > > > > >> > > > to the
> > > > > > >> > > > > > > > > > > > controller quorum rather than just to
> > > memory.
> > > > > But
> > > > > > >> we
> > > > > > >> > > > believe this
> > > > > > >> > > > > > > > > > should
> > > > > > >> > > > > > > > > > > > be pretty fast.
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > regards,
> > > > > > >> > > > > > > > > > > > Colin
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > Thanks!
> > > > > > >> > > > > > > > > > > > > -Jose
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > --
> > > > > > >> > > David Arthur
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Fri, Aug 23, 2019, at 06:24, Ryanne Dolan wrote:
> Colin, can you outline what specifically would be in scope for this KIP vs
> deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
> section? Is the idea to get to the bridge release with this KIP, and then
> go from there?
> 
> Ryanne
>

Hi Ryanne,

The goal for KIP-500 is to set out an overall vision for how we will remove ZooKeeper and transition to managing metadata via a controller quorum.

We will create follow-on KIPs that will lay out the specific details of each step.  

* A KIP for allowing kafka-configs.sh to change topic configurations without using ZooKeeper.  (It can already change broker configurations without ZK)

* A KIP for adding APIs to replace direct ZK access by the brokers.

* A KIP to describe Raft replication in Kafka, including the overall protocol, details of each RPC, etc.

* A KIP describing the controller changes, how metadata is stored, etc.

There may be other KIPs that we need (for example, if we find another tool that still has a hard ZK dependency), but that's the general idea.  KIP-500 is about the overall design-- the follow on KIPs are about the specific details.

best,
Colin


> 
> On Thu, Aug 22, 2019, 11:58 AM Colin McCabe <cm...@apache.org> wrote:
> 
> > On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> > > Thanks, Colin.  The changes you made to the KIP related to the bridge
> > > release help make it clearer.  I still have some confusion about the
> > phrase
> > > "The rolling upgrade from the bridge release will take several steps."
> > > This made me think you are talking about moving from the bridge release
> > to
> > > some other, newer, release that comes after the bridge release.  But I
> > > think what you are getting at is that the bridge release can be run with
> > or
> > > without Zookeeper -- when first upgrading to it Zookeeper remains in use,
> > > but then there is a transition that can be made to engage the warp
> > drive...
> > > I mean the Controller Quorum.  So maybe the phrase should be "The rolling
> > > upgrade through the bridge release -- starting with Zookeeper being in
> > use
> > > and ending with Zookeeper having been replaced by the Controller Quorum
> > --
> > > will take several steps."
> >
> > Hi Ron,
> >
> > To clarify, the bridge release will require ZooKeeper.  It will also not
> > support the controller quorum.  It's a bridge in the sense that you must
> > upgrade to a bridge release prior to upgrading to a ZK-less release.  I
> > added some more descriptive text to the bridge release paragraph--
> > hopefully this makes it clearer.
> >
> > best,
> > Colin
> >
> > >
> > > Do I understand it correctly, and might some change in phrasing or
> > > additional clarification help others avoid the same confusion I had?
> > >
> > > Ron
> > >
> > > On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> > > > > Hi Colin.  I like the concept of a "bridge release" for migrating
> > off of
> > > > > Zookeeper, but I worry that it may become a bottleneck if people
> > hesitate
> > > > > to replace Zookeeper -- they would be unable to adopt newer versions
> > of
> > > > > Kafka until taking (what feels to them like) a giant leap.  As an
> > > > example,
> > > > > assuming version 4.0.x of Kafka is the supported bridge release, I
> > would
> > > > > not be surprised if uptake of the 4.x release and the time-based
> > releases
> > > > > that follow it end up being much slower due to the perceived barrier.
> > > > >
> > > > > Any perceived barrier could be lowered if the 4.0.x release could
> > > > > optionally continue to use Zookeeper -- then the cutover would be two
> > > > > incremental steps (move to 4.0.x, then replace Zookeeper while
> > staying on
> > > > > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> > > > > Zookeeper in one fell swoop).
> > > >
> > > > Hi Ron,
> > > >
> > > > Just to clarify, the "bridge release" will continue to use ZooKeeper.
> > It
> > > > will not support running without ZooKeeper.  It is the releases that
> > follow
> > > > the bridge release that will remove ZooKeeper.
> > > >
> > > > Also, it's a bit unclear whether the bridge release would be 3.x or
> > 4.x,
> > > > or something to follow.  We do know that the bridge release can't be a
> > 2.x
> > > > release, since it requires at least one incompatible change, removing
> > > > --zookeeper options from all the shell scripts.  (Since we're doing
> > > > semantic versioning, any time we make an incompatible change, we bump
> > the
> > > > major version number.)
> > > >
> > > > In general, using two sources of metadata is a lot more complex and
> > > > error-prone than one.  A lot of the bugs and corner cases we have are
> > the
> > > > result of divergences between the controller and the state in
> > ZooKeeper.
> > > > Eliminating this divergence, and the split-brain scenarios it creates,
> > is a
> > > > major goal of this work.
> > > >
> > > > >
> > > > > Regardless of whether what I wrote above has merit or not, I think
> > the
> > > > KIP
> > > > > should be more explicit about what the upgrade constraints actually
> > are.
> > > > > Can the bridge release be adopted with Zookeeper remaining in place
> > and
> > > > > then cutting over as a second, follow-on step, or must the Controller
> > > > > Quorum nodes be started first and the bridge release cannot be used
> > with
> > > > > Zookeeper at all?
> > > >
> > > > As I mentioned above, the bridge release supports (indeed, requires)
> > > > ZooKeeper.  I have added a little more text about this to KIP-500 which
> > > > hopefully makes it clearer.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >  If the bridge release cannot be used with Zookeeper at
> > > > > all, then no version at or beyond the bridge release is available
> > > > > unless/until abandoning Zookeeper; if the bridge release can be used
> > with
> > > > > Zookeeper, then is it the only version that can be used with
> > Zookeeper,
> > > > or
> > > > > can Zookeeper be kept for additional releases if desired?
> > > > >
> > > > > Ron
> > > > >
> > > > > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hi Colin.  The diagram up at the top confused me -- specifically,
> > the
> > > > > > lines connecting the controller/active-controller to the brokers.
> > I
> > > > had
> > > > > > assumed the arrows on those lines represented the direction of data
> > > > flow,
> > > > > > but that is not the case; the arrows actually identify the target
> > of
> > > > the
> > > > > > action, and the non-arrowed end indicates the initiator of the
> > > > action.  For
> > > > > > example, the lines point from the controller to the brokers in the
> > > > "today"
> > > > > > section on the left to show that the controller pushes to the
> > brokers;
> > > > the
> > > > > > lines point from the brokers to the active-controller in the
> > "tomorrow"
> > > > > > section on the right to show that the brokers pull from the
> > > > > > active-controller.  As I said, this confused me because my gut
> > > > instinct was
> > > > > > to interpret the arrow as indicating the direction of data flow,
> > and
> > > > when I
> > > > > > look at the "tomorrow" picture on the right I initially thought
> > > > information
> > > > > > was moving from the brokers to the active-controller.  Did you
> > consider
> > > > > > drawing that picture with the arrows reversed in the "tomorrow"
> > side so
> > > > > > that the arrows represent the direction of data flow, and then add
> > the
> > > > > > labels "push" on the "today" side and "pull" on the "tomorrow"
> > side to
> > > > > > indicate who initiates the data flow?  It occurs to me that this
> > > > picture
> > > > > > may end up being widely distributed, so it might be in everyone's
> > > > interest
> > > > > > to proactively avoid any possible confusion by being more explicit.
> > > > > >
> > > > > > Minor corrections?
> > > > > > <<<In the current world, a broker which can contact ZooKeeper but
> > which
> > > > > > is partitioned from the active controller
> > > > > > >>>In the current world, a broker which can contact ZooKeeper but
> > which
> > > > > > is partitioned from the controller
> > > > > >
> > > > > > <<<Eventually, the controller will ask the broker to finally go
> > offline
> > > > > > >>>Eventually, the active controller will ask the broker to
> > finally go
> > > > > > offline
> > > > > >
> > > > > > <<<New versions of the clients should send these operations
> > directly to
> > > > > > the controller
> > > > > > >>>New versions of the clients should send these operations
> > directly to
> > > > > > the active controller
> > > > > >
> > > > > > <<<In the post-ZK world, the leader will make an RPC to the
> > controller
> > > > > > instead
> > > > > > >>>In the post-ZK world, the leader will make an RPC to the active
> > > > > > controller instead
> > > > > >
> > > > > > <<<For example, the brokers may need to forward their requests to
> > the
> > > > > > controller.
> > > > > > >>>For example, the brokers may need to forward their requests to
> > the
> > > > > > active controller.
> > > > > >
> > > > > > <<<The new controller will monitor ZooKeeper for legacy broker node
> > > > > > registrations
> > > > > > >>>The new (active) controller will monitor ZooKeeper for legacy
> > broker
> > > > > > node registrations
> > > > > >
> > > > > > Ron
> > > > > >
> > > > > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org>
> > > > wrote:
> > > > > >
> > > > > >> Hi all,
> > > > > >>
> > > > > >> The KIP has been out for a while, so I'm thinking about calling a
> > vote
> > > > > >> some time this week.
> > > > > >>
> > > > > >> best,
> > > > > >> Colin
> > > > > >>
> > > > > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > > > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > > > >> > > Thanks for the KIP, Colin. This looks great!
> > > > > >> > >
> > > > > >> > > I really like the idea of separating the Controller and Broker
> > > > JVMs.
> > > > > >> > >
> > > > > >> > > As you alluded to above, it might be nice to have a separate
> > > > > >> > > broker-registration API to avoid overloading the metadata
> > fetch
> > > > API.
> > > > > >> > >
> > > > > >> >
> > > > > >> > Hi David,
> > > > > >> >
> > > > > >> > Thanks for taking a look.
> > > > > >> >
> > > > > >> > I removed the sentence about MetadataFetch also serving as the
> > > > broker
> > > > > >> > registration API.  I think I agree that we will probably want a
> > > > > >> > separate RPC to fill this role.  We will have a follow-on KIP
> > that
> > > > will
> > > > > >> > go into more detail about metadata propagation and registration
> > in
> > > > the
> > > > > >> > post-ZK world.  That KIP will also have a full description of
> > the
> > > > > >> > registration RPC, etc.  For now, I think the important part for
> > > > KIP-500
> > > > > >> > is that the broker registers with the controller quorum.  On
> > > > > >> > registration, the controller quorum assigns it a new broker
> > epoch,
> > > > > >> > which can distinguish successive broker incarnations.
> > > > > >> >
> > > > > >> > >
> > > > > >> > > When a broker gets a metadata delta, will it be a sequence of
> > > > deltas
> > > > > >> since
> > > > > >> > > the last update or a cumulative delta since the last update?
> > > > > >> > >
> > > > > >> >
> > > > > >> > It will be a sequence of deltas.  Basically, the broker will be
> > > > reading
> > > > > >> > from the metadata log.
> > > > > >> >
> > > > > >> > >
> > > > > >> > > Will we include any kind of integrity check on the deltas to
> > > > ensure
> > > > > >> the brokers
> > > > > >> > > have applied them correctly? Perhaps this will be addressed in
> > > > one of
> > > > > >> the
> > > > > >> > > follow-on KIPs.
> > > > > >> > >
> > > > > >> >
> > > > > >> > In general, we will have checksums on the metadata that we
> > fetch.
> > > > This
> > > > > >> > is similar to how we have checksums on regular data.  Or if the
> > > > > >> > question is about catching logic errors in the metadata handling
> > > > code,
> > > > > >> > that sounds more like something that should be caught by test
> > cases.
> > > > > >> >
> > > > > >> > best,
> > > > > >> > Colin
> > > > > >> >
> > > > > >> >
> > > > > >> > > Thanks!
> > > > > >> > >
> > > > > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <
> > cmccabe@apache.org>
> > > > > >> wrote:
> > > > > >> > >
> > > > > >> > > > Hi Mickael,
> > > > > >> > > >
> > > > > >> > > > Thanks for taking a look.
> > > > > >> > > >
> > > > > >> > > > I don't think we want to support that kind of multi-tenancy
> > at
> > > > the
> > > > > >> > > > controller level.  If the cluster is small enough that we
> > want
> > > > to
> > > > > >> pack the
> > > > > >> > > > controller(s) with something else, we could run them
> > alongside
> > > > the
> > > > > >> brokers,
> > > > > >> > > > or possibly inside three of the broker JVMs.
> > > > > >> > > >
> > > > > >> > > > best,
> > > > > >> > > > Colin
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > > >> > > > > Thank Colin for kickstarting this initiative.
> > > > > >> > > > >
> > > > > >> > > > > Just one question.
> > > > > >> > > > > - A nice feature of Zookeeper is the ability to use
> > chroots
> > > > and
> > > > > >> have
> > > > > >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is
> > > > this
> > > > > >> > > > > something we should keep?
> > > > > >> > > > >
> > > > > >> > > > > Thanks
> > > > > >> > > > >
> > > > > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> > > > cmccabe@apache.org>
> > > > > >> wrote:
> > > > > >> > > > > >
> > > > > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > > >> > > > > > > Hi Colin,
> > > > > >> > > > > > >
> > > > > >> > > > > > > Thanks for the KIP.
> > > > > >> > > > > > >
> > > > > >> > > > > > > Currently ZooKeeper provides a convenient notification
> > > > > >> mechanism for
> > > > > >> > > > > > > knowing that broker and topic configuration has
> > changed.
> > > > While
> > > > > >> > > > KIP-500 does
> > > > > >> > > > > > > suggest that incremental metadata update is expected
> > to
> > > > come
> > > > > >> to
> > > > > >> > > > clients
> > > > > >> > > > > > > eventually, that would seem to imply that for some
> > number
> > > > of
> > > > > >> > > > releases there
> > > > > >> > > > > > > would be no equivalent mechanism for knowing about
> > config
> > > > > >> changes.
> > > > > >> > > > Is there
> > > > > >> > > > > > > any thinking at this point about how a similar
> > > > notification
> > > > > >> might be
> > > > > >> > > > > > > provided in the future?
> > > > > >> > > > > >
> > > > > >> > > > > > We could eventually have some inotify-like mechanism
> > where
> > > > > >> clients
> > > > > >> > > > could register interest in various types of events and got
> > > > notified
> > > > > >> when
> > > > > >> > > > they happened.  Reading the metadata log is conceptually
> > simple.
> > > > > >> The main
> > > > > >> > > > complexity would be in setting up an API that made sense and
> > > > that
> > > > > >> didn't
> > > > > >> > > > unduly constrain future implementations.  We'd have to think
> > > > > >> carefully
> > > > > >> > > > about what the real use-cases for this were, though.
> > > > > >> > > > > >
> > > > > >> > > > > > best,
> > > > > >> > > > > > Colin
> > > > > >> > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > Thanks,
> > > > > >> > > > > > >
> > > > > >> > > > > > > Tom
> > > > > >> > > > > > >
> > > > > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > > > >> > > > viktorsomogyi@gmail.com>
> > > > > >> > > > > > > wrote:
> > > > > >> > > > > > >
> > > > > >> > > > > > > > Hey Colin,
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > I think this is a long-awaited KIP, thanks for
> > driving
> > > > it.
> > > > > >> I'm
> > > > > >> > > > excited to
> > > > > >> > > > > > > > see this in Kafka once. I collected my questions
> > (and I
> > > > > >> accept the
> > > > > >> > > > "TBD"
> > > > > >> > > > > > > > answer as they might be a bit deep for this high
> > level
> > > > :) ).
> > > > > >> > > > > > > > 1.) Are there any specific reasons for the
> > Controller
> > > > just
> > > > > >> > > > periodically
> > > > > >> > > > > > > > persisting its state on disk periodically instead of
> > > > > >> > > > asynchronously with
> > > > > >> > > > > > > > every update? Wouldn't less frequent saves increase
> > the
> > > > > >> chance for
> > > > > >> > > > missing
> > > > > >> > > > > > > > a state change if the controller crashes between two
> > > > saves?
> > > > > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata
> > from
> > > > the
> > > > > >> follower
> > > > > >> > > > > > > > controllers? I assume that followers would have
> > > > up-to-date
> > > > > >> > > > information
> > > > > >> > > > > > > > therefore brokers could fetch from there in theory.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Thanks,
> > > > > >> > > > > > > > Viktor
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > > > >> > > > reluctanthero104@gmail.com>
> > > > > >> > > > > > > > wrote:
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> > > > > >> follow-up KIPs
> > > > > >> > > > sounds
> > > > > >> > > > > > > > like
> > > > > >> > > > > > > > > a good idea.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > > > > >> ismael@juma.me.uk>
> > > > > >> > > > wrote:
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > > Hi Boyang,
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > > > Yes, there will be several KIPs that will
> > discuss
> > > > the
> > > > > >> items you
> > > > > >> > > > > > > > describe
> > > > > >> > > > > > > > > in
> > > > > >> > > > > > > > > > detail. Colin, it may be helpful to make this
> > clear
> > > > in
> > > > > >> the KIP
> > > > > >> > > > 500
> > > > > >> > > > > > > > > > description.
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > > > Ismael
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > > > >> > > > reluctanthero104@gmail.com
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > > wrote:
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > > > > Thanks Colin for initiating this important
> > effort!
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > > > One question I have is whether we have a
> > session
> > > > > >> discussing
> > > > > >> > > > the
> > > > > >> > > > > > > > > > controller
> > > > > >> > > > > > > > > > > failover in the new architecture? I know we
> > are
> > > > using
> > > > > >> Raft
> > > > > >> > > > protocol
> > > > > >> > > > > > > > to
> > > > > >> > > > > > > > > > > failover, yet it's still valuable to discuss
> > the
> > > > > >> steps new
> > > > > >> > > > cluster is
> > > > > >> > > > > > > > > > going
> > > > > >> > > > > > > > > > > to take to reach the stable stage again, so
> > that
> > > > we
> > > > > >> could
> > > > > >> > > > easily
> > > > > >> > > > > > > > > measure
> > > > > >> > > > > > > > > > > the availability of the metadata servers.
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > > > Another suggestion I have is to write a
> > > > step-by-step
> > > > > >> design
> > > > > >> > > > doc like
> > > > > >> > > > > > > > > what
> > > > > >> > > > > > > > > > > we did in KIP-98
> > > > > >> > > > > > > > > > > <
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > >
> > > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > >> > > > > > > > > > > >,
> > > > > >> > > > > > > > > > > including the new request protocols and how
> > they
> > > > are
> > > > > >> > > > interacting in
> > > > > >> > > > > > > > the
> > > > > >> > > > > > > > > > new
> > > > > >> > > > > > > > > > > cluster. For a complicated change like this,
> > an
> > > > > >> > > > implementation design
> > > > > >> > > > > > > > > doc
> > > > > >> > > > > > > > > > > help a lot in the review process, otherwise
> > most
> > > > > >> discussions
> > > > > >> > > > we have
> > > > > >> > > > > > > > > will
> > > > > >> > > > > > > > > > > focus on high level and lose important
> > details as
> > > > we
> > > > > >> > > > discover them in
> > > > > >> > > > > > > > > the
> > > > > >> > > > > > > > > > > post-agreement phase.
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > > > Boyang
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > > > >> > > > cmccabe@apache.org>
> > > > > >> > > > > > > > > wrote:
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando
> > > > Garcia
> > > > > >> Sancio
> > > > > >> > > > wrote:
> > > > > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a
> > few
> > > > > >> comments
> > > > > >> > > > and
> > > > > >> > > > > > > > > questions.
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> > > > > >> mentioned the
> > > > > >> > > > > > > > LeaderAndIsr
> > > > > >> > > > > > > > > > and
> > > > > >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates
> > > > which
> > > > > >> the
> > > > > >> > > > controller
> > > > > >> > > > > > > > > > pushes,
> > > > > >> > > > > > > > > > > > such
> > > > > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata
> > messages".
> > > > Is
> > > > > >> your
> > > > > >> > > > thinking
> > > > > >> > > > > > > > that
> > > > > >> > > > > > > > > > we
> > > > > >> > > > > > > > > > > > will
> > > > > >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > > > >> > > > UpdateMetadata only
> > > > > >> > > > > > > > and
> > > > > >> > > > > > > > > > add
> > > > > >> > > > > > > > > > > > > topic configuration in this state?
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > Hi Jose,
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > Thanks for taking a look.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to
> > replace
> > > > both
> > > > > >> > > > > > > > > > LeaderAndIsrRequest
> > > > > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic
> > configurations
> > > > > >> would be
> > > > > >> > > > fetched
> > > > > >> > > > > > > > > along
> > > > > >> > > > > > > > > > > > with the other metadata.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > In the section "Broker Metadata
> > Management",
> > > > you
> > > > > >> mention
> > > > > >> > > > "Just
> > > > > >> > > > > > > > like
> > > > > >> > > > > > > > > > > with
> > > > > >> > > > > > > > > > > > a
> > > > > >> > > > > > > > > > > > > fetch request, the broker will track the
> > > > offset
> > > > > >> of the
> > > > > >> > > > last
> > > > > >> > > > > > > > updates
> > > > > >> > > > > > > > > > it
> > > > > >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> > > > > >> requires that
> > > > > >> > > > the
> > > > > >> > > > > > > > > followers
> > > > > >> > > > > > > > > > > > keep
> > > > > >> > > > > > > > > > > > > all of the log entries (term/epoch and
> > offset)
> > > > > >> that are
> > > > > >> > > > after the
> > > > > >> > > > > > > > > > > > > highwatermark. Any log entry before the
> > > > > >> highwatermark
> > > > > >> > > > can be
> > > > > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > > > > >> MetadataFetch API
> > > > > >> > > > to only
> > > > > >> > > > > > > > > return
> > > > > >> > > > > > > > > > > log
> > > > > >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike
> > the
> > > > Raft
> > > > > >> > > > replication API
> > > > > >> > > > > > > > > > which
> > > > > >> > > > > > > > > > > > > will replicate/fetch log entries after the
> > > > > >> highwatermark
> > > > > >> > > > for
> > > > > >> > > > > > > > > > consensus?
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> > > > > >> metadata
> > > > > >> > > > updates to
> > > > > >> > > > > > > > the
> > > > > >> > > > > > > > > > > > brokers until they've been stored on a
> > majority
> > > > of
> > > > > >> the
> > > > > >> > > > Raft nodes.
> > > > > >> > > > > > > > > The
> > > > > >> > > > > > > > > > > > most obvious way to do that, like you
> > > > mentioned, is
> > > > > >> to
> > > > > >> > > > have the
> > > > > >> > > > > > > > > brokers
> > > > > >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.
> > There
> > > > > >> might be
> > > > > >> > > > a more
> > > > > >> > > > > > > > > clever
> > > > > >> > > > > > > > > > > way
> > > > > >> > > > > > > > > > > > to do it by fetching the data, but not
> > having
> > > > the
> > > > > >> brokers
> > > > > >> > > > act on it
> > > > > >> > > > > > > > > > until
> > > > > >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's
> > worth
> > > > it
> > > > > >> or
> > > > > >> > > > not.  We'll
> > > > > >> > > > > > > > > > discuss
> > > > > >> > > > > > > > > > > > this more in a separate KIP that just
> > discusses
> > > > > >> just Raft.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > In section "Broker Metadata Management",
> > you
> > > > > >> mention "the
> > > > > >> > > > > > > > > controller
> > > > > >> > > > > > > > > > > will
> > > > > >> > > > > > > > > > > > > send a full metadata image rather than a
> > > > series of
> > > > > >> > > > deltas". This
> > > > > >> > > > > > > > > KIP
> > > > > >> > > > > > > > > > > > > doesn't go into the set of operations that
> > > > need
> > > > > >> to be
> > > > > >> > > > supported
> > > > > >> > > > > > > > on
> > > > > >> > > > > > > > > > top
> > > > > >> > > > > > > > > > > of
> > > > > >> > > > > > > > > > > > > Raft but it would be interested if this
> > "full
> > > > > >> metadata
> > > > > >> > > > image"
> > > > > >> > > > > > > > could
> > > > > >> > > > > > > > > > be
> > > > > >> > > > > > > > > > > > > express also as deltas. For example,
> > assuming
> > > > we
> > > > > >> are
> > > > > >> > > > replicating
> > > > > >> > > > > > > > a
> > > > > >> > > > > > > > > > map
> > > > > >> > > > > > > > > > > > this
> > > > > >> > > > > > > > > > > > > "full metadata image" could be a sequence
> > of
> > > > "put"
> > > > > >> > > > operations
> > > > > >> > > > > > > > > (znode
> > > > > >> > > > > > > > > > > > create
> > > > > >> > > > > > > > > > > > > to borrow ZK semantics).
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > The full image can definitely be expressed
> > as a
> > > > sum
> > > > > >> of
> > > > > >> > > > deltas.  At
> > > > > >> > > > > > > > > some
> > > > > >> > > > > > > > > > > > point, the number of deltas will get large
> > > > enough
> > > > > >> that
> > > > > >> > > > sending a
> > > > > >> > > > > > > > full
> > > > > >> > > > > > > > > > > image
> > > > > >> > > > > > > > > > > > is better, though.  One question that we're
> > > > still
> > > > > >> thinking
> > > > > >> > > > about is
> > > > > >> > > > > > > > > how
> > > > > >> > > > > > > > > > > > much of this can be shared with generic
> > Kafka
> > > > log
> > > > > >> code,
> > > > > >> > > > and how
> > > > > >> > > > > > > > much
> > > > > >> > > > > > > > > > > should
> > > > > >> > > > > > > > > > > > be different.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > In section "Broker Metadata Management",
> > you
> > > > > >> mention
> > > > > >> > > > "This
> > > > > >> > > > > > > > request
> > > > > >> > > > > > > > > > will
> > > > > >> > > > > > > > > > > > > double as a heartbeat, letting the
> > controller
> > > > > >> know that
> > > > > >> > > > the
> > > > > >> > > > > > > > broker
> > > > > >> > > > > > > > > is
> > > > > >> > > > > > > > > > > > > alive". In section "Broker State
> > Machine", you
> > > > > >> mention
> > > > > >> > > > "The
> > > > > >> > > > > > > > > > > MetadataFetch
> > > > > >> > > > > > > > > > > > > API serves as this registration
> > mechanism".
> > > > Does
> > > > > >> this
> > > > > >> > > > mean that
> > > > > >> > > > > > > > the
> > > > > >> > > > > > > > > > > > > MetadataFetch Request will optionally
> > include
> > > > > >> broker
> > > > > >> > > > > > > > configuration
> > > > > >> > > > > > > > > > > > > information?
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > I was originally thinking that the
> > > > > >> MetadataFetchRequest
> > > > > >> > > > should
> > > > > >> > > > > > > > > include
> > > > > >> > > > > > > > > > > > broker configuration information.  Thinking
> > > > about
> > > > > >> this
> > > > > >> > > > more, maybe
> > > > > >> > > > > > > > we
> > > > > >> > > > > > > > > > > > should just have a special registration RPC
> > that
> > > > > >> contains
> > > > > >> > > > that
> > > > > >> > > > > > > > > > > information,
> > > > > >> > > > > > > > > > > > to avoid sending it over the wire all the
> > time.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > Does this also mean that MetadataFetch
> > request
> > > > > >> will
> > > > > >> > > > result in
> > > > > >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > > > > >> replication
> > > > > >> > > > protocol
> > > > > >> > > > > > > > > before
> > > > > >> > > > > > > > > > > you
> > > > > >> > > > > > > > > > > > > can send the associated MetadataFetch
> > > > Response?
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > I think we should require the broker to be
> > out
> > > > of
> > > > > >> the
> > > > > >> > > > Offline state
> > > > > >> > > > > > > > > > > before
> > > > > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the
> > > > separate
> > > > > >> > > > registration
> > > > > >> > > > > > > > RPC
> > > > > >> > > > > > > > > > > > should have completed first.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > In section "Broker State", you mention
> > that a
> > > > > >> broker can
> > > > > >> > > > > > > > transition
> > > > > >> > > > > > > > > > to
> > > > > >> > > > > > > > > > > > > online after it is caught with the
> > metadata.
> > > > What
> > > > > >> do you
> > > > > >> > > > mean by
> > > > > >> > > > > > > > > > this?
> > > > > >> > > > > > > > > > > > > Metadata is always changing. How does the
> > > > broker
> > > > > >> know
> > > > > >> > > > that it is
> > > > > >> > > > > > > > > > caught
> > > > > >> > > > > > > > > > > > up
> > > > > >> > > > > > > > > > > > > since it doesn't participate in the
> > consensus
> > > > or
> > > > > >> the
> > > > > >> > > > advancement
> > > > > >> > > > > > > > of
> > > > > >> > > > > > > > > > the
> > > > > >> > > > > > > > > > > > > highwatermark?
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > That's a good point.  Being "caught up" is
> > > > somewhat
> > > > > >> of a
> > > > > >> > > > fuzzy
> > > > > >> > > > > > > > > concept
> > > > > >> > > > > > > > > > > > here, since the brokers do not participate
> > in
> > > > the
> > > > > >> metadata
> > > > > >> > > > > > > > consensus.
> > > > > >> > > > > > > > > > I
> > > > > >> > > > > > > > > > > > think ideally we would want to define it in
> > > > terms
> > > > > >> of time
> > > > > >> > > > ("the
> > > > > >> > > > > > > > > broker
> > > > > >> > > > > > > > > > > has
> > > > > >> > > > > > > > > > > > all the updates from the last 2 minutes",
> > for
> > > > > >> example.)
> > > > > >> > > > We should
> > > > > >> > > > > > > > > > spell
> > > > > >> > > > > > > > > > > > this out better in the KIP.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > In section "Start the controller quorum
> > > > nodes",
> > > > > >> you
> > > > > >> > > > mention "Once
> > > > > >> > > > > > > > > it
> > > > > >> > > > > > > > > > > has
> > > > > >> > > > > > > > > > > > > taken over the /controller node, the
> > active
> > > > > >> controller
> > > > > >> > > > will
> > > > > >> > > > > > > > proceed
> > > > > >> > > > > > > > > > to
> > > > > >> > > > > > > > > > > > load
> > > > > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will
> > write
> > > > out
> > > > > >> this
> > > > > >> > > > information
> > > > > >> > > > > > > > to
> > > > > >> > > > > > > > > > the
> > > > > >> > > > > > > > > > > > > quorum's metadata storage.  After this
> > point,
> > > > the
> > > > > >> > > > metadata quorum
> > > > > >> > > > > > > > > > will
> > > > > >> > > > > > > > > > > be
> > > > > >> > > > > > > > > > > > > the metadata store of record, rather than
> > the
> > > > > >> data in
> > > > > >> > > > ZooKeeper."
> > > > > >> > > > > > > > > > > During
> > > > > >> > > > > > > > > > > > > this migration do should we expect to
> > have a
> > > > > >> small period
> > > > > >> > > > > > > > > controller
> > > > > >> > > > > > > > > > > > > unavailability while the controller
> > replicas
> > > > this
> > > > > >> state
> > > > > >> > > > to all of
> > > > > >> > > > > > > > > the
> > > > > >> > > > > > > > > > > > raft
> > > > > >> > > > > > > > > > > > > nodes in the controller quorum and we
> > buffer
> > > > new
> > > > > >> > > > controller API
> > > > > >> > > > > > > > > > > requests?
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > Yes, the controller would be unavailable
> > during
> > > > this
> > > > > >> > > > time.  I don't
> > > > > >> > > > > > > > > > think
> > > > > >> > > > > > > > > > > > this will be that different from the current
> > > > period
> > > > > >> of
> > > > > >> > > > > > > > unavailability
> > > > > >> > > > > > > > > > > when
> > > > > >> > > > > > > > > > > > a new controller starts up and needs to
> > load the
> > > > > >> full
> > > > > >> > > > state from
> > > > > >> > > > > > > > ZK.
> > > > > >> > > > > > > > > > The
> > > > > >> > > > > > > > > > > > main difference is that in this period, we'd
> > > > have
> > > > > >> to write
> > > > > >> > > > to the
> > > > > >> > > > > > > > > > > > controller quorum rather than just to
> > memory.
> > > > But
> > > > > >> we
> > > > > >> > > > believe this
> > > > > >> > > > > > > > > > should
> > > > > >> > > > > > > > > > > > be pretty fast.
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > regards,
> > > > > >> > > > > > > > > > > > Colin
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > Thanks!
> > > > > >> > > > > > > > > > > > > -Jose
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > --
> > > > > >> > > David Arthur
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ryanne Dolan <ry...@gmail.com>.
Colin, can you outline what specifically would be in scope for this KIP vs
deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
section? Is the idea to get to the bridge release with this KIP, and then
go from there?

Ryanne

On Thu, Aug 22, 2019, 11:58 AM Colin McCabe <cm...@apache.org> wrote:

> On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> > Thanks, Colin.  The changes you made to the KIP related to the bridge
> > release help make it clearer.  I still have some confusion about the
> phrase
> > "The rolling upgrade from the bridge release will take several steps."
> > This made me think you are talking about moving from the bridge release
> to
> > some other, newer, release that comes after the bridge release.  But I
> > think what you are getting at is that the bridge release can be run with
> or
> > without Zookeeper -- when first upgrading to it Zookeeper remains in use,
> > but then there is a transition that can be made to engage the warp
> drive...
> > I mean the Controller Quorum.  So maybe the phrase should be "The rolling
> > upgrade through the bridge release -- starting with Zookeeper being in
> use
> > and ending with Zookeeper having been replaced by the Controller Quorum
> --
> > will take several steps."
>
> Hi Ron,
>
> To clarify, the bridge release will require ZooKeeper.  It will also not
> support the controller quorum.  It's a bridge in the sense that you must
> upgrade to a bridge release prior to upgrading to a ZK-less release.  I
> added some more descriptive text to the bridge release paragraph--
> hopefully this makes it clearer.
>
> best,
> Colin
>
> >
> > Do I understand it correctly, and might some change in phrasing or
> > additional clarification help others avoid the same confusion I had?
> >
> > Ron
> >
> > On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> > > > Hi Colin.  I like the concept of a "bridge release" for migrating
> off of
> > > > Zookeeper, but I worry that it may become a bottleneck if people
> hesitate
> > > > to replace Zookeeper -- they would be unable to adopt newer versions
> of
> > > > Kafka until taking (what feels to them like) a giant leap.  As an
> > > example,
> > > > assuming version 4.0.x of Kafka is the supported bridge release, I
> would
> > > > not be surprised if uptake of the 4.x release and the time-based
> releases
> > > > that follow it end up being much slower due to the perceived barrier.
> > > >
> > > > Any perceived barrier could be lowered if the 4.0.x release could
> > > > optionally continue to use Zookeeper -- then the cutover would be two
> > > > incremental steps (move to 4.0.x, then replace Zookeeper while
> staying on
> > > > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> > > > Zookeeper in one fell swoop).
> > >
> > > Hi Ron,
> > >
> > > Just to clarify, the "bridge release" will continue to use ZooKeeper.
> It
> > > will not support running without ZooKeeper.  It is the releases that
> follow
> > > the bridge release that will remove ZooKeeper.
> > >
> > > Also, it's a bit unclear whether the bridge release would be 3.x or
> 4.x,
> > > or something to follow.  We do know that the bridge release can't be a
> 2.x
> > > release, since it requires at least one incompatible change, removing
> > > --zookeeper options from all the shell scripts.  (Since we're doing
> > > semantic versioning, any time we make an incompatible change, we bump
> the
> > > major version number.)
> > >
> > > In general, using two sources of metadata is a lot more complex and
> > > error-prone than one.  A lot of the bugs and corner cases we have are
> the
> > > result of divergences between the controller and the state in
> ZooKeeper.
> > > Eliminating this divergence, and the split-brain scenarios it creates,
> is a
> > > major goal of this work.
> > >
> > > >
> > > > Regardless of whether what I wrote above has merit or not, I think
> the
> > > KIP
> > > > should be more explicit about what the upgrade constraints actually
> are.
> > > > Can the bridge release be adopted with Zookeeper remaining in place
> and
> > > > then cutting over as a second, follow-on step, or must the Controller
> > > > Quorum nodes be started first and the bridge release cannot be used
> with
> > > > Zookeeper at all?
> > >
> > > As I mentioned above, the bridge release supports (indeed, requires)
> > > ZooKeeper.  I have added a little more text about this to KIP-500 which
> > > hopefully makes it clearer.
> > >
> > > best,
> > > Colin
> > >
> > > >  If the bridge release cannot be used with Zookeeper at
> > > > all, then no version at or beyond the bridge release is available
> > > > unless/until abandoning Zookeeper; if the bridge release can be used
> with
> > > > Zookeeper, then is it the only version that can be used with
> Zookeeper,
> > > or
> > > > can Zookeeper be kept for additional releases if desired?
> > > >
> > > > Ron
> > > >
> > > > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi Colin.  The diagram up at the top confused me -- specifically,
> the
> > > > > lines connecting the controller/active-controller to the brokers.
> I
> > > had
> > > > > assumed the arrows on those lines represented the direction of data
> > > flow,
> > > > > but that is not the case; the arrows actually identify the target
> of
> > > the
> > > > > action, and the non-arrowed end indicates the initiator of the
> > > action.  For
> > > > > example, the lines point from the controller to the brokers in the
> > > "today"
> > > > > section on the left to show that the controller pushes to the
> brokers;
> > > the
> > > > > lines point from the brokers to the active-controller in the
> "tomorrow"
> > > > > section on the right to show that the brokers pull from the
> > > > > active-controller.  As I said, this confused me because my gut
> > > instinct was
> > > > > to interpret the arrow as indicating the direction of data flow,
> and
> > > when I
> > > > > look at the "tomorrow" picture on the right I initially thought
> > > information
> > > > > was moving from the brokers to the active-controller.  Did you
> consider
> > > > > drawing that picture with the arrows reversed in the "tomorrow"
> side so
> > > > > that the arrows represent the direction of data flow, and then add
> the
> > > > > labels "push" on the "today" side and "pull" on the "tomorrow"
> side to
> > > > > indicate who initiates the data flow?  It occurs to me that this
> > > picture
> > > > > may end up being widely distributed, so it might be in everyone's
> > > interest
> > > > > to proactively avoid any possible confusion by being more explicit.
> > > > >
> > > > > Minor corrections?
> > > > > <<<In the current world, a broker which can contact ZooKeeper but
> which
> > > > > is partitioned from the active controller
> > > > > >>>In the current world, a broker which can contact ZooKeeper but
> which
> > > > > is partitioned from the controller
> > > > >
> > > > > <<<Eventually, the controller will ask the broker to finally go
> offline
> > > > > >>>Eventually, the active controller will ask the broker to
> finally go
> > > > > offline
> > > > >
> > > > > <<<New versions of the clients should send these operations
> directly to
> > > > > the controller
> > > > > >>>New versions of the clients should send these operations
> directly to
> > > > > the active controller
> > > > >
> > > > > <<<In the post-ZK world, the leader will make an RPC to the
> controller
> > > > > instead
> > > > > >>>In the post-ZK world, the leader will make an RPC to the active
> > > > > controller instead
> > > > >
> > > > > <<<For example, the brokers may need to forward their requests to
> the
> > > > > controller.
> > > > > >>>For example, the brokers may need to forward their requests to
> the
> > > > > active controller.
> > > > >
> > > > > <<<The new controller will monitor ZooKeeper for legacy broker node
> > > > > registrations
> > > > > >>>The new (active) controller will monitor ZooKeeper for legacy
> broker
> > > > > node registrations
> > > > >
> > > > > Ron
> > > > >
> > > > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org>
> > > wrote:
> > > > >
> > > > >> Hi all,
> > > > >>
> > > > >> The KIP has been out for a while, so I'm thinking about calling a
> vote
> > > > >> some time this week.
> > > > >>
> > > > >> best,
> > > > >> Colin
> > > > >>
> > > > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > > >> > > Thanks for the KIP, Colin. This looks great!
> > > > >> > >
> > > > >> > > I really like the idea of separating the Controller and Broker
> > > JVMs.
> > > > >> > >
> > > > >> > > As you alluded to above, it might be nice to have a separate
> > > > >> > > broker-registration API to avoid overloading the metadata
> fetch
> > > API.
> > > > >> > >
> > > > >> >
> > > > >> > Hi David,
> > > > >> >
> > > > >> > Thanks for taking a look.
> > > > >> >
> > > > >> > I removed the sentence about MetadataFetch also serving as the
> > > broker
> > > > >> > registration API.  I think I agree that we will probably want a
> > > > >> > separate RPC to fill this role.  We will have a follow-on KIP
> that
> > > will
> > > > >> > go into more detail about metadata propagation and registration
> in
> > > the
> > > > >> > post-ZK world.  That KIP will also have a full description of
> the
> > > > >> > registration RPC, etc.  For now, I think the important part for
> > > KIP-500
> > > > >> > is that the broker registers with the controller quorum.  On
> > > > >> > registration, the controller quorum assigns it a new broker
> epoch,
> > > > >> > which can distinguish successive broker incarnations.
> > > > >> >
> > > > >> > >
> > > > >> > > When a broker gets a metadata delta, will it be a sequence of
> > > deltas
> > > > >> since
> > > > >> > > the last update or a cumulative delta since the last update?
> > > > >> > >
> > > > >> >
> > > > >> > It will be a sequence of deltas.  Basically, the broker will be
> > > reading
> > > > >> > from the metadata log.
> > > > >> >
> > > > >> > >
> > > > >> > > Will we include any kind of integrity check on the deltas to
> > > ensure
> > > > >> the brokers
> > > > >> > > have applied them correctly? Perhaps this will be addressed in
> > > one of
> > > > >> the
> > > > >> > > follow-on KIPs.
> > > > >> > >
> > > > >> >
> > > > >> > In general, we will have checksums on the metadata that we
> fetch.
> > > This
> > > > >> > is similar to how we have checksums on regular data.  Or if the
> > > > >> > question is about catching logic errors in the metadata handling
> > > code,
> > > > >> > that sounds more like something that should be caught by test
> cases.
> > > > >> >
> > > > >> > best,
> > > > >> > Colin
> > > > >> >
> > > > >> >
> > > > >> > > Thanks!
> > > > >> > >
> > > > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <
> cmccabe@apache.org>
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Hi Mickael,
> > > > >> > > >
> > > > >> > > > Thanks for taking a look.
> > > > >> > > >
> > > > >> > > > I don't think we want to support that kind of multi-tenancy
> at
> > > the
> > > > >> > > > controller level.  If the cluster is small enough that we
> want
> > > to
> > > > >> pack the
> > > > >> > > > controller(s) with something else, we could run them
> alongside
> > > the
> > > > >> brokers,
> > > > >> > > > or possibly inside three of the broker JVMs.
> > > > >> > > >
> > > > >> > > > best,
> > > > >> > > > Colin
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > >> > > > > Thank Colin for kickstarting this initiative.
> > > > >> > > > >
> > > > >> > > > > Just one question.
> > > > >> > > > > - A nice feature of Zookeeper is the ability to use
> chroots
> > > and
> > > > >> have
> > > > >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is
> > > this
> > > > >> > > > > something we should keep?
> > > > >> > > > >
> > > > >> > > > > Thanks
> > > > >> > > > >
> > > > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> > > cmccabe@apache.org>
> > > > >> wrote:
> > > > >> > > > > >
> > > > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > >> > > > > > > Hi Colin,
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks for the KIP.
> > > > >> > > > > > >
> > > > >> > > > > > > Currently ZooKeeper provides a convenient notification
> > > > >> mechanism for
> > > > >> > > > > > > knowing that broker and topic configuration has
> changed.
> > > While
> > > > >> > > > KIP-500 does
> > > > >> > > > > > > suggest that incremental metadata update is expected
> to
> > > come
> > > > >> to
> > > > >> > > > clients
> > > > >> > > > > > > eventually, that would seem to imply that for some
> number
> > > of
> > > > >> > > > releases there
> > > > >> > > > > > > would be no equivalent mechanism for knowing about
> config
> > > > >> changes.
> > > > >> > > > Is there
> > > > >> > > > > > > any thinking at this point about how a similar
> > > notification
> > > > >> might be
> > > > >> > > > > > > provided in the future?
> > > > >> > > > > >
> > > > >> > > > > > We could eventually have some inotify-like mechanism
> where
> > > > >> clients
> > > > >> > > > could register interest in various types of events and got
> > > notified
> > > > >> when
> > > > >> > > > they happened.  Reading the metadata log is conceptually
> simple.
> > > > >> The main
> > > > >> > > > complexity would be in setting up an API that made sense and
> > > that
> > > > >> didn't
> > > > >> > > > unduly constrain future implementations.  We'd have to think
> > > > >> carefully
> > > > >> > > > about what the real use-cases for this were, though.
> > > > >> > > > > >
> > > > >> > > > > > best,
> > > > >> > > > > > Colin
> > > > >> > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks,
> > > > >> > > > > > >
> > > > >> > > > > > > Tom
> > > > >> > > > > > >
> > > > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > > >> > > > viktorsomogyi@gmail.com>
> > > > >> > > > > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hey Colin,
> > > > >> > > > > > > >
> > > > >> > > > > > > > I think this is a long-awaited KIP, thanks for
> driving
> > > it.
> > > > >> I'm
> > > > >> > > > excited to
> > > > >> > > > > > > > see this in Kafka once. I collected my questions
> (and I
> > > > >> accept the
> > > > >> > > > "TBD"
> > > > >> > > > > > > > answer as they might be a bit deep for this high
> level
> > > :) ).
> > > > >> > > > > > > > 1.) Are there any specific reasons for the
> Controller
> > > just
> > > > >> > > > periodically
> > > > >> > > > > > > > persisting its state on disk periodically instead of
> > > > >> > > > asynchronously with
> > > > >> > > > > > > > every update? Wouldn't less frequent saves increase
> the
> > > > >> chance for
> > > > >> > > > missing
> > > > >> > > > > > > > a state change if the controller crashes between two
> > > saves?
> > > > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata
> from
> > > the
> > > > >> follower
> > > > >> > > > > > > > controllers? I assume that followers would have
> > > up-to-date
> > > > >> > > > information
> > > > >> > > > > > > > therefore brokers could fetch from there in theory.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks,
> > > > >> > > > > > > > Viktor
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > > >> > > > reluctanthero104@gmail.com>
> > > > >> > > > > > > > wrote:
> > > > >> > > > > > > >
> > > > >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> > > > >> follow-up KIPs
> > > > >> > > > sounds
> > > > >> > > > > > > > like
> > > > >> > > > > > > > > a good idea.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > > > >> ismael@juma.me.uk>
> > > > >> > > > wrote:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > > Hi Boyang,
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > Yes, there will be several KIPs that will
> discuss
> > > the
> > > > >> items you
> > > > >> > > > > > > > describe
> > > > >> > > > > > > > > in
> > > > >> > > > > > > > > > detail. Colin, it may be helpful to make this
> clear
> > > in
> > > > >> the KIP
> > > > >> > > > 500
> > > > >> > > > > > > > > > description.
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > Ismael
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > > >> > > > reluctanthero104@gmail.com
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > > wrote:
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > > > > Thanks Colin for initiating this important
> effort!
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > One question I have is whether we have a
> session
> > > > >> discussing
> > > > >> > > > the
> > > > >> > > > > > > > > > controller
> > > > >> > > > > > > > > > > failover in the new architecture? I know we
> are
> > > using
> > > > >> Raft
> > > > >> > > > protocol
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > > > failover, yet it's still valuable to discuss
> the
> > > > >> steps new
> > > > >> > > > cluster is
> > > > >> > > > > > > > > > going
> > > > >> > > > > > > > > > > to take to reach the stable stage again, so
> that
> > > we
> > > > >> could
> > > > >> > > > easily
> > > > >> > > > > > > > > measure
> > > > >> > > > > > > > > > > the availability of the metadata servers.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > Another suggestion I have is to write a
> > > step-by-step
> > > > >> design
> > > > >> > > > doc like
> > > > >> > > > > > > > > what
> > > > >> > > > > > > > > > > we did in KIP-98
> > > > >> > > > > > > > > > > <
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > >
> > > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >> > > > > > > > > > > >,
> > > > >> > > > > > > > > > > including the new request protocols and how
> they
> > > are
> > > > >> > > > interacting in
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > new
> > > > >> > > > > > > > > > > cluster. For a complicated change like this,
> an
> > > > >> > > > implementation design
> > > > >> > > > > > > > > doc
> > > > >> > > > > > > > > > > help a lot in the review process, otherwise
> most
> > > > >> discussions
> > > > >> > > > we have
> > > > >> > > > > > > > > will
> > > > >> > > > > > > > > > > focus on high level and lose important
> details as
> > > we
> > > > >> > > > discover them in
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > > post-agreement phase.
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > Boyang
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > > >> > > > cmccabe@apache.org>
> > > > >> > > > > > > > > wrote:
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando
> > > Garcia
> > > > >> Sancio
> > > > >> > > > wrote:
> > > > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a
> few
> > > > >> comments
> > > > >> > > > and
> > > > >> > > > > > > > > questions.
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> > > > >> mentioned the
> > > > >> > > > > > > > LeaderAndIsr
> > > > >> > > > > > > > > > and
> > > > >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates
> > > which
> > > > >> the
> > > > >> > > > controller
> > > > >> > > > > > > > > > pushes,
> > > > >> > > > > > > > > > > > such
> > > > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata
> messages".
> > > Is
> > > > >> your
> > > > >> > > > thinking
> > > > >> > > > > > > > that
> > > > >> > > > > > > > > > we
> > > > >> > > > > > > > > > > > will
> > > > >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > > >> > > > UpdateMetadata only
> > > > >> > > > > > > > and
> > > > >> > > > > > > > > > add
> > > > >> > > > > > > > > > > > > topic configuration in this state?
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Hi Jose,
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Thanks for taking a look.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to
> replace
> > > both
> > > > >> > > > > > > > > > LeaderAndIsrRequest
> > > > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic
> configurations
> > > > >> would be
> > > > >> > > > fetched
> > > > >> > > > > > > > > along
> > > > >> > > > > > > > > > > > with the other metadata.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In the section "Broker Metadata
> Management",
> > > you
> > > > >> mention
> > > > >> > > > "Just
> > > > >> > > > > > > > like
> > > > >> > > > > > > > > > > with
> > > > >> > > > > > > > > > > > a
> > > > >> > > > > > > > > > > > > fetch request, the broker will track the
> > > offset
> > > > >> of the
> > > > >> > > > last
> > > > >> > > > > > > > updates
> > > > >> > > > > > > > > > it
> > > > >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> > > > >> requires that
> > > > >> > > > the
> > > > >> > > > > > > > > followers
> > > > >> > > > > > > > > > > > keep
> > > > >> > > > > > > > > > > > > all of the log entries (term/epoch and
> offset)
> > > > >> that are
> > > > >> > > > after the
> > > > >> > > > > > > > > > > > > highwatermark. Any log entry before the
> > > > >> highwatermark
> > > > >> > > > can be
> > > > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > > > >> MetadataFetch API
> > > > >> > > > to only
> > > > >> > > > > > > > > return
> > > > >> > > > > > > > > > > log
> > > > >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike
> the
> > > Raft
> > > > >> > > > replication API
> > > > >> > > > > > > > > > which
> > > > >> > > > > > > > > > > > > will replicate/fetch log entries after the
> > > > >> highwatermark
> > > > >> > > > for
> > > > >> > > > > > > > > > consensus?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> > > > >> metadata
> > > > >> > > > updates to
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > > > brokers until they've been stored on a
> majority
> > > of
> > > > >> the
> > > > >> > > > Raft nodes.
> > > > >> > > > > > > > > The
> > > > >> > > > > > > > > > > > most obvious way to do that, like you
> > > mentioned, is
> > > > >> to
> > > > >> > > > have the
> > > > >> > > > > > > > > brokers
> > > > >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.
> There
> > > > >> might be
> > > > >> > > > a more
> > > > >> > > > > > > > > clever
> > > > >> > > > > > > > > > > way
> > > > >> > > > > > > > > > > > to do it by fetching the data, but not
> having
> > > the
> > > > >> brokers
> > > > >> > > > act on it
> > > > >> > > > > > > > > > until
> > > > >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's
> worth
> > > it
> > > > >> or
> > > > >> > > > not.  We'll
> > > > >> > > > > > > > > > discuss
> > > > >> > > > > > > > > > > > this more in a separate KIP that just
> discusses
> > > > >> just Raft.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Broker Metadata Management",
> you
> > > > >> mention "the
> > > > >> > > > > > > > > controller
> > > > >> > > > > > > > > > > will
> > > > >> > > > > > > > > > > > > send a full metadata image rather than a
> > > series of
> > > > >> > > > deltas". This
> > > > >> > > > > > > > > KIP
> > > > >> > > > > > > > > > > > > doesn't go into the set of operations that
> > > need
> > > > >> to be
> > > > >> > > > supported
> > > > >> > > > > > > > on
> > > > >> > > > > > > > > > top
> > > > >> > > > > > > > > > > of
> > > > >> > > > > > > > > > > > > Raft but it would be interested if this
> "full
> > > > >> metadata
> > > > >> > > > image"
> > > > >> > > > > > > > could
> > > > >> > > > > > > > > > be
> > > > >> > > > > > > > > > > > > express also as deltas. For example,
> assuming
> > > we
> > > > >> are
> > > > >> > > > replicating
> > > > >> > > > > > > > a
> > > > >> > > > > > > > > > map
> > > > >> > > > > > > > > > > > this
> > > > >> > > > > > > > > > > > > "full metadata image" could be a sequence
> of
> > > "put"
> > > > >> > > > operations
> > > > >> > > > > > > > > (znode
> > > > >> > > > > > > > > > > > create
> > > > >> > > > > > > > > > > > > to borrow ZK semantics).
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > The full image can definitely be expressed
> as a
> > > sum
> > > > >> of
> > > > >> > > > deltas.  At
> > > > >> > > > > > > > > some
> > > > >> > > > > > > > > > > > point, the number of deltas will get large
> > > enough
> > > > >> that
> > > > >> > > > sending a
> > > > >> > > > > > > > full
> > > > >> > > > > > > > > > > image
> > > > >> > > > > > > > > > > > is better, though.  One question that we're
> > > still
> > > > >> thinking
> > > > >> > > > about is
> > > > >> > > > > > > > > how
> > > > >> > > > > > > > > > > > much of this can be shared with generic
> Kafka
> > > log
> > > > >> code,
> > > > >> > > > and how
> > > > >> > > > > > > > much
> > > > >> > > > > > > > > > > should
> > > > >> > > > > > > > > > > > be different.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Broker Metadata Management",
> you
> > > > >> mention
> > > > >> > > > "This
> > > > >> > > > > > > > request
> > > > >> > > > > > > > > > will
> > > > >> > > > > > > > > > > > > double as a heartbeat, letting the
> controller
> > > > >> know that
> > > > >> > > > the
> > > > >> > > > > > > > broker
> > > > >> > > > > > > > > is
> > > > >> > > > > > > > > > > > > alive". In section "Broker State
> Machine", you
> > > > >> mention
> > > > >> > > > "The
> > > > >> > > > > > > > > > > MetadataFetch
> > > > >> > > > > > > > > > > > > API serves as this registration
> mechanism".
> > > Does
> > > > >> this
> > > > >> > > > mean that
> > > > >> > > > > > > > the
> > > > >> > > > > > > > > > > > > MetadataFetch Request will optionally
> include
> > > > >> broker
> > > > >> > > > > > > > configuration
> > > > >> > > > > > > > > > > > > information?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > I was originally thinking that the
> > > > >> MetadataFetchRequest
> > > > >> > > > should
> > > > >> > > > > > > > > include
> > > > >> > > > > > > > > > > > broker configuration information.  Thinking
> > > about
> > > > >> this
> > > > >> > > > more, maybe
> > > > >> > > > > > > > we
> > > > >> > > > > > > > > > > > should just have a special registration RPC
> that
> > > > >> contains
> > > > >> > > > that
> > > > >> > > > > > > > > > > information,
> > > > >> > > > > > > > > > > > to avoid sending it over the wire all the
> time.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > Does this also mean that MetadataFetch
> request
> > > > >> will
> > > > >> > > > result in
> > > > >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > > > >> replication
> > > > >> > > > protocol
> > > > >> > > > > > > > > before
> > > > >> > > > > > > > > > > you
> > > > >> > > > > > > > > > > > > can send the associated MetadataFetch
> > > Response?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > I think we should require the broker to be
> out
> > > of
> > > > >> the
> > > > >> > > > Offline state
> > > > >> > > > > > > > > > > before
> > > > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the
> > > separate
> > > > >> > > > registration
> > > > >> > > > > > > > RPC
> > > > >> > > > > > > > > > > > should have completed first.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Broker State", you mention
> that a
> > > > >> broker can
> > > > >> > > > > > > > transition
> > > > >> > > > > > > > > > to
> > > > >> > > > > > > > > > > > > online after it is caught with the
> metadata.
> > > What
> > > > >> do you
> > > > >> > > > mean by
> > > > >> > > > > > > > > > this?
> > > > >> > > > > > > > > > > > > Metadata is always changing. How does the
> > > broker
> > > > >> know
> > > > >> > > > that it is
> > > > >> > > > > > > > > > caught
> > > > >> > > > > > > > > > > > up
> > > > >> > > > > > > > > > > > > since it doesn't participate in the
> consensus
> > > or
> > > > >> the
> > > > >> > > > advancement
> > > > >> > > > > > > > of
> > > > >> > > > > > > > > > the
> > > > >> > > > > > > > > > > > > highwatermark?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > That's a good point.  Being "caught up" is
> > > somewhat
> > > > >> of a
> > > > >> > > > fuzzy
> > > > >> > > > > > > > > concept
> > > > >> > > > > > > > > > > > here, since the brokers do not participate
> in
> > > the
> > > > >> metadata
> > > > >> > > > > > > > consensus.
> > > > >> > > > > > > > > > I
> > > > >> > > > > > > > > > > > think ideally we would want to define it in
> > > terms
> > > > >> of time
> > > > >> > > > ("the
> > > > >> > > > > > > > > broker
> > > > >> > > > > > > > > > > has
> > > > >> > > > > > > > > > > > all the updates from the last 2 minutes",
> for
> > > > >> example.)
> > > > >> > > > We should
> > > > >> > > > > > > > > > spell
> > > > >> > > > > > > > > > > > this out better in the KIP.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > In section "Start the controller quorum
> > > nodes",
> > > > >> you
> > > > >> > > > mention "Once
> > > > >> > > > > > > > > it
> > > > >> > > > > > > > > > > has
> > > > >> > > > > > > > > > > > > taken over the /controller node, the
> active
> > > > >> controller
> > > > >> > > > will
> > > > >> > > > > > > > proceed
> > > > >> > > > > > > > > > to
> > > > >> > > > > > > > > > > > load
> > > > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will
> write
> > > out
> > > > >> this
> > > > >> > > > information
> > > > >> > > > > > > > to
> > > > >> > > > > > > > > > the
> > > > >> > > > > > > > > > > > > quorum's metadata storage.  After this
> point,
> > > the
> > > > >> > > > metadata quorum
> > > > >> > > > > > > > > > will
> > > > >> > > > > > > > > > > be
> > > > >> > > > > > > > > > > > > the metadata store of record, rather than
> the
> > > > >> data in
> > > > >> > > > ZooKeeper."
> > > > >> > > > > > > > > > > During
> > > > >> > > > > > > > > > > > > this migration do should we expect to
> have a
> > > > >> small period
> > > > >> > > > > > > > > controller
> > > > >> > > > > > > > > > > > > unavailability while the controller
> replicas
> > > this
> > > > >> state
> > > > >> > > > to all of
> > > > >> > > > > > > > > the
> > > > >> > > > > > > > > > > > raft
> > > > >> > > > > > > > > > > > > nodes in the controller quorum and we
> buffer
> > > new
> > > > >> > > > controller API
> > > > >> > > > > > > > > > > requests?
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > Yes, the controller would be unavailable
> during
> > > this
> > > > >> > > > time.  I don't
> > > > >> > > > > > > > > > think
> > > > >> > > > > > > > > > > > this will be that different from the current
> > > period
> > > > >> of
> > > > >> > > > > > > > unavailability
> > > > >> > > > > > > > > > > when
> > > > >> > > > > > > > > > > > a new controller starts up and needs to
> load the
> > > > >> full
> > > > >> > > > state from
> > > > >> > > > > > > > ZK.
> > > > >> > > > > > > > > > The
> > > > >> > > > > > > > > > > > main difference is that in this period, we'd
> > > have
> > > > >> to write
> > > > >> > > > to the
> > > > >> > > > > > > > > > > > controller quorum rather than just to
> memory.
> > > But
> > > > >> we
> > > > >> > > > believe this
> > > > >> > > > > > > > > > should
> > > > >> > > > > > > > > > > > be pretty fast.
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > regards,
> > > > >> > > > > > > > > > > > Colin
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > Thanks!
> > > > >> > > > > > > > > > > > > -Jose
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> > >
> > > > >> > > --
> > > > >> > > David Arthur
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> Thanks, Colin.  The changes you made to the KIP related to the bridge
> release help make it clearer.  I still have some confusion about the phrase
> "The rolling upgrade from the bridge release will take several steps."
> This made me think you are talking about moving from the bridge release to
> some other, newer, release that comes after the bridge release.  But I
> think what you are getting at is that the bridge release can be run with or
> without Zookeeper -- when first upgrading to it Zookeeper remains in use,
> but then there is a transition that can be made to engage the warp drive...
> I mean the Controller Quorum.  So maybe the phrase should be "The rolling
> upgrade through the bridge release -- starting with Zookeeper being in use
> and ending with Zookeeper having been replaced by the Controller Quorum --
> will take several steps."

Hi Ron,

To clarify, the bridge release will require ZooKeeper.  It will also not support the controller quorum.  It's a bridge in the sense that you must upgrade to a bridge release prior to upgrading to a ZK-less release.  I added some more descriptive text to the bridge release paragraph-- hopefully this makes it clearer.

best,
Colin

> 
> Do I understand it correctly, and might some change in phrasing or
> additional clarification help others avoid the same confusion I had?
> 
> Ron
> 
> On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> > > Hi Colin.  I like the concept of a "bridge release" for migrating off of
> > > Zookeeper, but I worry that it may become a bottleneck if people hesitate
> > > to replace Zookeeper -- they would be unable to adopt newer versions of
> > > Kafka until taking (what feels to them like) a giant leap.  As an
> > example,
> > > assuming version 4.0.x of Kafka is the supported bridge release, I  would
> > > not be surprised if uptake of the 4.x release and the time-based releases
> > > that follow it end up being much slower due to the perceived barrier.
> > >
> > > Any perceived barrier could be lowered if the 4.0.x release could
> > > optionally continue to use Zookeeper -- then the cutover would be two
> > > incremental steps (move to 4.0.x, then replace Zookeeper while staying on
> > > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> > > Zookeeper in one fell swoop).
> >
> > Hi Ron,
> >
> > Just to clarify, the "bridge release" will continue to use ZooKeeper.  It
> > will not support running without ZooKeeper.  It is the releases that follow
> > the bridge release that will remove ZooKeeper.
> >
> > Also, it's a bit unclear whether the bridge release would be 3.x or 4.x,
> > or something to follow.  We do know that the bridge release can't be a 2.x
> > release, since it requires at least one incompatible change, removing
> > --zookeeper options from all the shell scripts.  (Since we're doing
> > semantic versioning, any time we make an incompatible change, we bump the
> > major version number.)
> >
> > In general, using two sources of metadata is a lot more complex and
> > error-prone than one.  A lot of the bugs and corner cases we have are the
> > result of divergences between the controller and the state in ZooKeeper.
> > Eliminating this divergence, and the split-brain scenarios it creates, is a
> > major goal of this work.
> >
> > >
> > > Regardless of whether what I wrote above has merit or not, I think the
> > KIP
> > > should be more explicit about what the upgrade constraints actually are.
> > > Can the bridge release be adopted with Zookeeper remaining in place and
> > > then cutting over as a second, follow-on step, or must the Controller
> > > Quorum nodes be started first and the bridge release cannot be used with
> > > Zookeeper at all?
> >
> > As I mentioned above, the bridge release supports (indeed, requires)
> > ZooKeeper.  I have added a little more text about this to KIP-500 which
> > hopefully makes it clearer.
> >
> > best,
> > Colin
> >
> > >  If the bridge release cannot be used with Zookeeper at
> > > all, then no version at or beyond the bridge release is available
> > > unless/until abandoning Zookeeper; if the bridge release can be used with
> > > Zookeeper, then is it the only version that can be used with Zookeeper,
> > or
> > > can Zookeeper be kept for additional releases if desired?
> > >
> > > Ron
> > >
> > > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com>
> > wrote:
> > >
> > > > Hi Colin.  The diagram up at the top confused me -- specifically, the
> > > > lines connecting the controller/active-controller to the brokers.  I
> > had
> > > > assumed the arrows on those lines represented the direction of data
> > flow,
> > > > but that is not the case; the arrows actually identify the target of
> > the
> > > > action, and the non-arrowed end indicates the initiator of the
> > action.  For
> > > > example, the lines point from the controller to the brokers in the
> > "today"
> > > > section on the left to show that the controller pushes to the brokers;
> > the
> > > > lines point from the brokers to the active-controller in the "tomorrow"
> > > > section on the right to show that the brokers pull from the
> > > > active-controller.  As I said, this confused me because my gut
> > instinct was
> > > > to interpret the arrow as indicating the direction of data flow, and
> > when I
> > > > look at the "tomorrow" picture on the right I initially thought
> > information
> > > > was moving from the brokers to the active-controller.  Did you consider
> > > > drawing that picture with the arrows reversed in the "tomorrow" side so
> > > > that the arrows represent the direction of data flow, and then add the
> > > > labels "push" on the "today" side and "pull" on the "tomorrow" side to
> > > > indicate who initiates the data flow?  It occurs to me that this
> > picture
> > > > may end up being widely distributed, so it might be in everyone's
> > interest
> > > > to proactively avoid any possible confusion by being more explicit.
> > > >
> > > > Minor corrections?
> > > > <<<In the current world, a broker which can contact ZooKeeper but which
> > > > is partitioned from the active controller
> > > > >>>In the current world, a broker which can contact ZooKeeper but which
> > > > is partitioned from the controller
> > > >
> > > > <<<Eventually, the controller will ask the broker to finally go offline
> > > > >>>Eventually, the active controller will ask the broker to finally go
> > > > offline
> > > >
> > > > <<<New versions of the clients should send these operations directly to
> > > > the controller
> > > > >>>New versions of the clients should send these operations directly to
> > > > the active controller
> > > >
> > > > <<<In the post-ZK world, the leader will make an RPC to the controller
> > > > instead
> > > > >>>In the post-ZK world, the leader will make an RPC to the active
> > > > controller instead
> > > >
> > > > <<<For example, the brokers may need to forward their requests to the
> > > > controller.
> > > > >>>For example, the brokers may need to forward their requests to the
> > > > active controller.
> > > >
> > > > <<<The new controller will monitor ZooKeeper for legacy broker node
> > > > registrations
> > > > >>>The new (active) controller will monitor ZooKeeper for legacy broker
> > > > node registrations
> > > >
> > > > Ron
> > > >
> > > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> The KIP has been out for a while, so I'm thinking about calling a vote
> > > >> some time this week.
> > > >>
> > > >> best,
> > > >> Colin
> > > >>
> > > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > >> > > Thanks for the KIP, Colin. This looks great!
> > > >> > >
> > > >> > > I really like the idea of separating the Controller and Broker
> > JVMs.
> > > >> > >
> > > >> > > As you alluded to above, it might be nice to have a separate
> > > >> > > broker-registration API to avoid overloading the metadata fetch
> > API.
> > > >> > >
> > > >> >
> > > >> > Hi David,
> > > >> >
> > > >> > Thanks for taking a look.
> > > >> >
> > > >> > I removed the sentence about MetadataFetch also serving as the
> > broker
> > > >> > registration API.  I think I agree that we will probably want a
> > > >> > separate RPC to fill this role.  We will have a follow-on KIP that
> > will
> > > >> > go into more detail about metadata propagation and registration in
> > the
> > > >> > post-ZK world.  That KIP will also have a full description of the
> > > >> > registration RPC, etc.  For now, I think the important part for
> > KIP-500
> > > >> > is that the broker registers with the controller quorum.  On
> > > >> > registration, the controller quorum assigns it a new broker epoch,
> > > >> > which can distinguish successive broker incarnations.
> > > >> >
> > > >> > >
> > > >> > > When a broker gets a metadata delta, will it be a sequence of
> > deltas
> > > >> since
> > > >> > > the last update or a cumulative delta since the last update?
> > > >> > >
> > > >> >
> > > >> > It will be a sequence of deltas.  Basically, the broker will be
> > reading
> > > >> > from the metadata log.
> > > >> >
> > > >> > >
> > > >> > > Will we include any kind of integrity check on the deltas to
> > ensure
> > > >> the brokers
> > > >> > > have applied them correctly? Perhaps this will be addressed in
> > one of
> > > >> the
> > > >> > > follow-on KIPs.
> > > >> > >
> > > >> >
> > > >> > In general, we will have checksums on the metadata that we fetch.
> > This
> > > >> > is similar to how we have checksums on regular data.  Or if the
> > > >> > question is about catching logic errors in the metadata handling
> > code,
> > > >> > that sounds more like something that should be caught by test cases.
> > > >> >
> > > >> > best,
> > > >> > Colin
> > > >> >
> > > >> >
> > > >> > > Thanks!
> > > >> > >
> > > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
> > > >> wrote:
> > > >> > >
> > > >> > > > Hi Mickael,
> > > >> > > >
> > > >> > > > Thanks for taking a look.
> > > >> > > >
> > > >> > > > I don't think we want to support that kind of multi-tenancy at
> > the
> > > >> > > > controller level.  If the cluster is small enough that we want
> > to
> > > >> pack the
> > > >> > > > controller(s) with something else, we could run them alongside
> > the
> > > >> brokers,
> > > >> > > > or possibly inside three of the broker JVMs.
> > > >> > > >
> > > >> > > > best,
> > > >> > > > Colin
> > > >> > > >
> > > >> > > >
> > > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > >> > > > > Thank Colin for kickstarting this initiative.
> > > >> > > > >
> > > >> > > > > Just one question.
> > > >> > > > > - A nice feature of Zookeeper is the ability to use chroots
> > and
> > > >> have
> > > >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is
> > this
> > > >> > > > > something we should keep?
> > > >> > > > >
> > > >> > > > > Thanks
> > > >> > > > >
> > > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> > cmccabe@apache.org>
> > > >> wrote:
> > > >> > > > > >
> > > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > >> > > > > > > Hi Colin,
> > > >> > > > > > >
> > > >> > > > > > > Thanks for the KIP.
> > > >> > > > > > >
> > > >> > > > > > > Currently ZooKeeper provides a convenient notification
> > > >> mechanism for
> > > >> > > > > > > knowing that broker and topic configuration has changed.
> > While
> > > >> > > > KIP-500 does
> > > >> > > > > > > suggest that incremental metadata update is expected to
> > come
> > > >> to
> > > >> > > > clients
> > > >> > > > > > > eventually, that would seem to imply that for some number
> > of
> > > >> > > > releases there
> > > >> > > > > > > would be no equivalent mechanism for knowing about config
> > > >> changes.
> > > >> > > > Is there
> > > >> > > > > > > any thinking at this point about how a similar
> > notification
> > > >> might be
> > > >> > > > > > > provided in the future?
> > > >> > > > > >
> > > >> > > > > > We could eventually have some inotify-like mechanism where
> > > >> clients
> > > >> > > > could register interest in various types of events and got
> > notified
> > > >> when
> > > >> > > > they happened.  Reading the metadata log is conceptually simple.
> > > >> The main
> > > >> > > > complexity would be in setting up an API that made sense and
> > that
> > > >> didn't
> > > >> > > > unduly constrain future implementations.  We'd have to think
> > > >> carefully
> > > >> > > > about what the real use-cases for this were, though.
> > > >> > > > > >
> > > >> > > > > > best,
> > > >> > > > > > Colin
> > > >> > > > > >
> > > >> > > > > > >
> > > >> > > > > > > Thanks,
> > > >> > > > > > >
> > > >> > > > > > > Tom
> > > >> > > > > > >
> > > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > >> > > > viktorsomogyi@gmail.com>
> > > >> > > > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hey Colin,
> > > >> > > > > > > >
> > > >> > > > > > > > I think this is a long-awaited KIP, thanks for driving
> > it.
> > > >> I'm
> > > >> > > > excited to
> > > >> > > > > > > > see this in Kafka once. I collected my questions (and I
> > > >> accept the
> > > >> > > > "TBD"
> > > >> > > > > > > > answer as they might be a bit deep for this high level
> > :) ).
> > > >> > > > > > > > 1.) Are there any specific reasons for the Controller
> > just
> > > >> > > > periodically
> > > >> > > > > > > > persisting its state on disk periodically instead of
> > > >> > > > asynchronously with
> > > >> > > > > > > > every update? Wouldn't less frequent saves increase the
> > > >> chance for
> > > >> > > > missing
> > > >> > > > > > > > a state change if the controller crashes between two
> > saves?
> > > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from
> > the
> > > >> follower
> > > >> > > > > > > > controllers? I assume that followers would have
> > up-to-date
> > > >> > > > information
> > > >> > > > > > > > therefore brokers could fetch from there in theory.
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > > Viktor
> > > >> > > > > > > >
> > > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > >> > > > reluctanthero104@gmail.com>
> > > >> > > > > > > > wrote:
> > > >> > > > > > > >
> > > >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> > > >> follow-up KIPs
> > > >> > > > sounds
> > > >> > > > > > > > like
> > > >> > > > > > > > > a good idea.
> > > >> > > > > > > > >
> > > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > > >> ismael@juma.me.uk>
> > > >> > > > wrote:
> > > >> > > > > > > > >
> > > >> > > > > > > > > > Hi Boyang,
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > Yes, there will be several KIPs that will discuss
> > the
> > > >> items you
> > > >> > > > > > > > describe
> > > >> > > > > > > > > in
> > > >> > > > > > > > > > detail. Colin, it may be helpful to make this clear
> > in
> > > >> the KIP
> > > >> > > > 500
> > > >> > > > > > > > > > description.
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > Ismael
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > >> > > > reluctanthero104@gmail.com
> > > >> > > > > > > > >
> > > >> > > > > > > > > > wrote:
> > > >> > > > > > > > > >
> > > >> > > > > > > > > > > Thanks Colin for initiating this important effort!
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > One question I have is whether we have a session
> > > >> discussing
> > > >> > > > the
> > > >> > > > > > > > > > controller
> > > >> > > > > > > > > > > failover in the new architecture? I know we are
> > using
> > > >> Raft
> > > >> > > > protocol
> > > >> > > > > > > > to
> > > >> > > > > > > > > > > failover, yet it's still valuable to discuss the
> > > >> steps new
> > > >> > > > cluster is
> > > >> > > > > > > > > > going
> > > >> > > > > > > > > > > to take to reach the stable stage again, so that
> > we
> > > >> could
> > > >> > > > easily
> > > >> > > > > > > > > measure
> > > >> > > > > > > > > > > the availability of the metadata servers.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > Another suggestion I have is to write a
> > step-by-step
> > > >> design
> > > >> > > > doc like
> > > >> > > > > > > > > what
> > > >> > > > > > > > > > > we did in KIP-98
> > > >> > > > > > > > > > > <
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > >
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >> > > > > > > > > > > >,
> > > >> > > > > > > > > > > including the new request protocols and how they
> > are
> > > >> > > > interacting in
> > > >> > > > > > > > the
> > > >> > > > > > > > > > new
> > > >> > > > > > > > > > > cluster. For a complicated change like this, an
> > > >> > > > implementation design
> > > >> > > > > > > > > doc
> > > >> > > > > > > > > > > help a lot in the review process, otherwise most
> > > >> discussions
> > > >> > > > we have
> > > >> > > > > > > > > will
> > > >> > > > > > > > > > > focus on high level and lose important details as
> > we
> > > >> > > > discover them in
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > > post-agreement phase.
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > Boyang
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > >> > > > cmccabe@apache.org>
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando
> > Garcia
> > > >> Sancio
> > > >> > > > wrote:
> > > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
> > > >> comments
> > > >> > > > and
> > > >> > > > > > > > > questions.
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> > > >> mentioned the
> > > >> > > > > > > > LeaderAndIsr
> > > >> > > > > > > > > > and
> > > >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates
> > which
> > > >> the
> > > >> > > > controller
> > > >> > > > > > > > > > pushes,
> > > >> > > > > > > > > > > > such
> > > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages".
> > Is
> > > >> your
> > > >> > > > thinking
> > > >> > > > > > > > that
> > > >> > > > > > > > > > we
> > > >> > > > > > > > > > > > will
> > > >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > >> > > > UpdateMetadata only
> > > >> > > > > > > > and
> > > >> > > > > > > > > > add
> > > >> > > > > > > > > > > > > topic configuration in this state?
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Hi Jose,
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Thanks for taking a look.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to replace
> > both
> > > >> > > > > > > > > > LeaderAndIsrRequest
> > > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
> > > >> would be
> > > >> > > > fetched
> > > >> > > > > > > > > along
> > > >> > > > > > > > > > > > with the other metadata.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In the section "Broker Metadata Management",
> > you
> > > >> mention
> > > >> > > > "Just
> > > >> > > > > > > > like
> > > >> > > > > > > > > > > with
> > > >> > > > > > > > > > > > a
> > > >> > > > > > > > > > > > > fetch request, the broker will track the
> > offset
> > > >> of the
> > > >> > > > last
> > > >> > > > > > > > updates
> > > >> > > > > > > > > > it
> > > >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> > > >> requires that
> > > >> > > > the
> > > >> > > > > > > > > followers
> > > >> > > > > > > > > > > > keep
> > > >> > > > > > > > > > > > > all of the log entries (term/epoch and offset)
> > > >> that are
> > > >> > > > after the
> > > >> > > > > > > > > > > > > highwatermark. Any log entry before the
> > > >> highwatermark
> > > >> > > > can be
> > > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > > >> MetadataFetch API
> > > >> > > > to only
> > > >> > > > > > > > > return
> > > >> > > > > > > > > > > log
> > > >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike the
> > Raft
> > > >> > > > replication API
> > > >> > > > > > > > > > which
> > > >> > > > > > > > > > > > > will replicate/fetch log entries after the
> > > >> highwatermark
> > > >> > > > for
> > > >> > > > > > > > > > consensus?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> > > >> metadata
> > > >> > > > updates to
> > > >> > > > > > > > the
> > > >> > > > > > > > > > > > brokers until they've been stored on a majority
> > of
> > > >> the
> > > >> > > > Raft nodes.
> > > >> > > > > > > > > The
> > > >> > > > > > > > > > > > most obvious way to do that, like you
> > mentioned, is
> > > >> to
> > > >> > > > have the
> > > >> > > > > > > > > brokers
> > > >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
> > > >> might be
> > > >> > > > a more
> > > >> > > > > > > > > clever
> > > >> > > > > > > > > > > way
> > > >> > > > > > > > > > > > to do it by fetching the data, but not having
> > the
> > > >> brokers
> > > >> > > > act on it
> > > >> > > > > > > > > > until
> > > >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth
> > it
> > > >> or
> > > >> > > > not.  We'll
> > > >> > > > > > > > > > discuss
> > > >> > > > > > > > > > > > this more in a separate KIP that just discusses
> > > >> just Raft.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > > >> mention "the
> > > >> > > > > > > > > controller
> > > >> > > > > > > > > > > will
> > > >> > > > > > > > > > > > > send a full metadata image rather than a
> > series of
> > > >> > > > deltas". This
> > > >> > > > > > > > > KIP
> > > >> > > > > > > > > > > > > doesn't go into the set of operations that
> > need
> > > >> to be
> > > >> > > > supported
> > > >> > > > > > > > on
> > > >> > > > > > > > > > top
> > > >> > > > > > > > > > > of
> > > >> > > > > > > > > > > > > Raft but it would be interested if this "full
> > > >> metadata
> > > >> > > > image"
> > > >> > > > > > > > could
> > > >> > > > > > > > > > be
> > > >> > > > > > > > > > > > > express also as deltas. For example, assuming
> > we
> > > >> are
> > > >> > > > replicating
> > > >> > > > > > > > a
> > > >> > > > > > > > > > map
> > > >> > > > > > > > > > > > this
> > > >> > > > > > > > > > > > > "full metadata image" could be a sequence of
> > "put"
> > > >> > > > operations
> > > >> > > > > > > > > (znode
> > > >> > > > > > > > > > > > create
> > > >> > > > > > > > > > > > > to borrow ZK semantics).
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > The full image can definitely be expressed as a
> > sum
> > > >> of
> > > >> > > > deltas.  At
> > > >> > > > > > > > > some
> > > >> > > > > > > > > > > > point, the number of deltas will get large
> > enough
> > > >> that
> > > >> > > > sending a
> > > >> > > > > > > > full
> > > >> > > > > > > > > > > image
> > > >> > > > > > > > > > > > is better, though.  One question that we're
> > still
> > > >> thinking
> > > >> > > > about is
> > > >> > > > > > > > > how
> > > >> > > > > > > > > > > > much of this can be shared with generic Kafka
> > log
> > > >> code,
> > > >> > > > and how
> > > >> > > > > > > > much
> > > >> > > > > > > > > > > should
> > > >> > > > > > > > > > > > be different.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > > >> mention
> > > >> > > > "This
> > > >> > > > > > > > request
> > > >> > > > > > > > > > will
> > > >> > > > > > > > > > > > > double as a heartbeat, letting the controller
> > > >> know that
> > > >> > > > the
> > > >> > > > > > > > broker
> > > >> > > > > > > > > is
> > > >> > > > > > > > > > > > > alive". In section "Broker State Machine", you
> > > >> mention
> > > >> > > > "The
> > > >> > > > > > > > > > > MetadataFetch
> > > >> > > > > > > > > > > > > API serves as this registration mechanism".
> > Does
> > > >> this
> > > >> > > > mean that
> > > >> > > > > > > > the
> > > >> > > > > > > > > > > > > MetadataFetch Request will optionally include
> > > >> broker
> > > >> > > > > > > > configuration
> > > >> > > > > > > > > > > > > information?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > I was originally thinking that the
> > > >> MetadataFetchRequest
> > > >> > > > should
> > > >> > > > > > > > > include
> > > >> > > > > > > > > > > > broker configuration information.  Thinking
> > about
> > > >> this
> > > >> > > > more, maybe
> > > >> > > > > > > > we
> > > >> > > > > > > > > > > > should just have a special registration RPC that
> > > >> contains
> > > >> > > > that
> > > >> > > > > > > > > > > information,
> > > >> > > > > > > > > > > > to avoid sending it over the wire all the time.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > > Does this also mean that MetadataFetch request
> > > >> will
> > > >> > > > result in
> > > >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > > >> replication
> > > >> > > > protocol
> > > >> > > > > > > > > before
> > > >> > > > > > > > > > > you
> > > >> > > > > > > > > > > > > can send the associated MetadataFetch
> > Response?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > I think we should require the broker to be out
> > of
> > > >> the
> > > >> > > > Offline state
> > > >> > > > > > > > > > > before
> > > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the
> > separate
> > > >> > > > registration
> > > >> > > > > > > > RPC
> > > >> > > > > > > > > > > > should have completed first.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Broker State", you mention that a
> > > >> broker can
> > > >> > > > > > > > transition
> > > >> > > > > > > > > > to
> > > >> > > > > > > > > > > > > online after it is caught with the metadata.
> > What
> > > >> do you
> > > >> > > > mean by
> > > >> > > > > > > > > > this?
> > > >> > > > > > > > > > > > > Metadata is always changing. How does the
> > broker
> > > >> know
> > > >> > > > that it is
> > > >> > > > > > > > > > caught
> > > >> > > > > > > > > > > > up
> > > >> > > > > > > > > > > > > since it doesn't participate in the consensus
> > or
> > > >> the
> > > >> > > > advancement
> > > >> > > > > > > > of
> > > >> > > > > > > > > > the
> > > >> > > > > > > > > > > > > highwatermark?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > That's a good point.  Being "caught up" is
> > somewhat
> > > >> of a
> > > >> > > > fuzzy
> > > >> > > > > > > > > concept
> > > >> > > > > > > > > > > > here, since the brokers do not participate in
> > the
> > > >> metadata
> > > >> > > > > > > > consensus.
> > > >> > > > > > > > > > I
> > > >> > > > > > > > > > > > think ideally we would want to define it in
> > terms
> > > >> of time
> > > >> > > > ("the
> > > >> > > > > > > > > broker
> > > >> > > > > > > > > > > has
> > > >> > > > > > > > > > > > all the updates from the last 2 minutes", for
> > > >> example.)
> > > >> > > > We should
> > > >> > > > > > > > > > spell
> > > >> > > > > > > > > > > > this out better in the KIP.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > In section "Start the controller quorum
> > nodes",
> > > >> you
> > > >> > > > mention "Once
> > > >> > > > > > > > > it
> > > >> > > > > > > > > > > has
> > > >> > > > > > > > > > > > > taken over the /controller node, the active
> > > >> controller
> > > >> > > > will
> > > >> > > > > > > > proceed
> > > >> > > > > > > > > > to
> > > >> > > > > > > > > > > > load
> > > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will write
> > out
> > > >> this
> > > >> > > > information
> > > >> > > > > > > > to
> > > >> > > > > > > > > > the
> > > >> > > > > > > > > > > > > quorum's metadata storage.  After this point,
> > the
> > > >> > > > metadata quorum
> > > >> > > > > > > > > > will
> > > >> > > > > > > > > > > be
> > > >> > > > > > > > > > > > > the metadata store of record, rather than the
> > > >> data in
> > > >> > > > ZooKeeper."
> > > >> > > > > > > > > > > During
> > > >> > > > > > > > > > > > > this migration do should we expect to have a
> > > >> small period
> > > >> > > > > > > > > controller
> > > >> > > > > > > > > > > > > unavailability while the controller replicas
> > this
> > > >> state
> > > >> > > > to all of
> > > >> > > > > > > > > the
> > > >> > > > > > > > > > > > raft
> > > >> > > > > > > > > > > > > nodes in the controller quorum and we buffer
> > new
> > > >> > > > controller API
> > > >> > > > > > > > > > > requests?
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > Yes, the controller would be unavailable during
> > this
> > > >> > > > time.  I don't
> > > >> > > > > > > > > > think
> > > >> > > > > > > > > > > > this will be that different from the current
> > period
> > > >> of
> > > >> > > > > > > > unavailability
> > > >> > > > > > > > > > > when
> > > >> > > > > > > > > > > > a new controller starts up and needs to load the
> > > >> full
> > > >> > > > state from
> > > >> > > > > > > > ZK.
> > > >> > > > > > > > > > The
> > > >> > > > > > > > > > > > main difference is that in this period, we'd
> > have
> > > >> to write
> > > >> > > > to the
> > > >> > > > > > > > > > > > controller quorum rather than just to memory.
> > But
> > > >> we
> > > >> > > > believe this
> > > >> > > > > > > > > > should
> > > >> > > > > > > > > > > > be pretty fast.
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > regards,
> > > >> > > > > > > > > > > > Colin
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > Thanks!
> > > >> > > > > > > > > > > > > -Jose
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> > >
> > > >> > > --
> > > >> > > David Arthur
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ron Dagostino <rn...@gmail.com>.
Thanks, Colin.  The changes you made to the KIP related to the bridge
release help make it clearer.  I still have some confusion about the phrase
"The rolling upgrade from the bridge release will take several steps."
This made me think you are talking about moving from the bridge release to
some other, newer, release that comes after the bridge release.  But I
think what you are getting at is that the bridge release can be run with or
without Zookeeper -- when first upgrading to it Zookeeper remains in use,
but then there is a transition that can be made to engage the warp drive...
I mean the Controller Quorum.  So maybe the phrase should be "The rolling
upgrade through the bridge release -- starting with Zookeeper being in use
and ending with Zookeeper having been replaced by the Controller Quorum --
will take several steps."

Do I understand it correctly, and might some change in phrasing or
additional clarification help others avoid the same confusion I had?

Ron

On Wed, Aug 21, 2019 at 2:31 PM Colin McCabe <cm...@apache.org> wrote:

> On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> > Hi Colin.  I like the concept of a "bridge release" for migrating off of
> > Zookeeper, but I worry that it may become a bottleneck if people hesitate
> > to replace Zookeeper -- they would be unable to adopt newer versions of
> > Kafka until taking (what feels to them like) a giant leap.  As an
> example,
> > assuming version 4.0.x of Kafka is the supported bridge release, I  would
> > not be surprised if uptake of the 4.x release and the time-based releases
> > that follow it end up being much slower due to the perceived barrier.
> >
> > Any perceived barrier could be lowered if the 4.0.x release could
> > optionally continue to use Zookeeper -- then the cutover would be two
> > incremental steps (move to 4.0.x, then replace Zookeeper while staying on
> > 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> > Zookeeper in one fell swoop).
>
> Hi Ron,
>
> Just to clarify, the "bridge release" will continue to use ZooKeeper.  It
> will not support running without ZooKeeper.  It is the releases that follow
> the bridge release that will remove ZooKeeper.
>
> Also, it's a bit unclear whether the bridge release would be 3.x or 4.x,
> or something to follow.  We do know that the bridge release can't be a 2.x
> release, since it requires at least one incompatible change, removing
> --zookeeper options from all the shell scripts.  (Since we're doing
> semantic versioning, any time we make an incompatible change, we bump the
> major version number.)
>
> In general, using two sources of metadata is a lot more complex and
> error-prone than one.  A lot of the bugs and corner cases we have are the
> result of divergences between the controller and the state in ZooKeeper.
> Eliminating this divergence, and the split-brain scenarios it creates, is a
> major goal of this work.
>
> >
> > Regardless of whether what I wrote above has merit or not, I think the
> KIP
> > should be more explicit about what the upgrade constraints actually are.
> > Can the bridge release be adopted with Zookeeper remaining in place and
> > then cutting over as a second, follow-on step, or must the Controller
> > Quorum nodes be started first and the bridge release cannot be used with
> > Zookeeper at all?
>
> As I mentioned above, the bridge release supports (indeed, requires)
> ZooKeeper.  I have added a little more text about this to KIP-500 which
> hopefully makes it clearer.
>
> best,
> Colin
>
> >  If the bridge release cannot be used with Zookeeper at
> > all, then no version at or beyond the bridge release is available
> > unless/until abandoning Zookeeper; if the bridge release can be used with
> > Zookeeper, then is it the only version that can be used with Zookeeper,
> or
> > can Zookeeper be kept for additional releases if desired?
> >
> > Ron
> >
> > On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com>
> wrote:
> >
> > > Hi Colin.  The diagram up at the top confused me -- specifically, the
> > > lines connecting the controller/active-controller to the brokers.  I
> had
> > > assumed the arrows on those lines represented the direction of data
> flow,
> > > but that is not the case; the arrows actually identify the target of
> the
> > > action, and the non-arrowed end indicates the initiator of the
> action.  For
> > > example, the lines point from the controller to the brokers in the
> "today"
> > > section on the left to show that the controller pushes to the brokers;
> the
> > > lines point from the brokers to the active-controller in the "tomorrow"
> > > section on the right to show that the brokers pull from the
> > > active-controller.  As I said, this confused me because my gut
> instinct was
> > > to interpret the arrow as indicating the direction of data flow, and
> when I
> > > look at the "tomorrow" picture on the right I initially thought
> information
> > > was moving from the brokers to the active-controller.  Did you consider
> > > drawing that picture with the arrows reversed in the "tomorrow" side so
> > > that the arrows represent the direction of data flow, and then add the
> > > labels "push" on the "today" side and "pull" on the "tomorrow" side to
> > > indicate who initiates the data flow?  It occurs to me that this
> picture
> > > may end up being widely distributed, so it might be in everyone's
> interest
> > > to proactively avoid any possible confusion by being more explicit.
> > >
> > > Minor corrections?
> > > <<<In the current world, a broker which can contact ZooKeeper but which
> > > is partitioned from the active controller
> > > >>>In the current world, a broker which can contact ZooKeeper but which
> > > is partitioned from the controller
> > >
> > > <<<Eventually, the controller will ask the broker to finally go offline
> > > >>>Eventually, the active controller will ask the broker to finally go
> > > offline
> > >
> > > <<<New versions of the clients should send these operations directly to
> > > the controller
> > > >>>New versions of the clients should send these operations directly to
> > > the active controller
> > >
> > > <<<In the post-ZK world, the leader will make an RPC to the controller
> > > instead
> > > >>>In the post-ZK world, the leader will make an RPC to the active
> > > controller instead
> > >
> > > <<<For example, the brokers may need to forward their requests to the
> > > controller.
> > > >>>For example, the brokers may need to forward their requests to the
> > > active controller.
> > >
> > > <<<The new controller will monitor ZooKeeper for legacy broker node
> > > registrations
> > > >>>The new (active) controller will monitor ZooKeeper for legacy broker
> > > node registrations
> > >
> > > Ron
> > >
> > > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org>
> wrote:
> > >
> > >> Hi all,
> > >>
> > >> The KIP has been out for a while, so I'm thinking about calling a vote
> > >> some time this week.
> > >>
> > >> best,
> > >> Colin
> > >>
> > >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > >> > > Thanks for the KIP, Colin. This looks great!
> > >> > >
> > >> > > I really like the idea of separating the Controller and Broker
> JVMs.
> > >> > >
> > >> > > As you alluded to above, it might be nice to have a separate
> > >> > > broker-registration API to avoid overloading the metadata fetch
> API.
> > >> > >
> > >> >
> > >> > Hi David,
> > >> >
> > >> > Thanks for taking a look.
> > >> >
> > >> > I removed the sentence about MetadataFetch also serving as the
> broker
> > >> > registration API.  I think I agree that we will probably want a
> > >> > separate RPC to fill this role.  We will have a follow-on KIP that
> will
> > >> > go into more detail about metadata propagation and registration in
> the
> > >> > post-ZK world.  That KIP will also have a full description of the
> > >> > registration RPC, etc.  For now, I think the important part for
> KIP-500
> > >> > is that the broker registers with the controller quorum.  On
> > >> > registration, the controller quorum assigns it a new broker epoch,
> > >> > which can distinguish successive broker incarnations.
> > >> >
> > >> > >
> > >> > > When a broker gets a metadata delta, will it be a sequence of
> deltas
> > >> since
> > >> > > the last update or a cumulative delta since the last update?
> > >> > >
> > >> >
> > >> > It will be a sequence of deltas.  Basically, the broker will be
> reading
> > >> > from the metadata log.
> > >> >
> > >> > >
> > >> > > Will we include any kind of integrity check on the deltas to
> ensure
> > >> the brokers
> > >> > > have applied them correctly? Perhaps this will be addressed in
> one of
> > >> the
> > >> > > follow-on KIPs.
> > >> > >
> > >> >
> > >> > In general, we will have checksums on the metadata that we fetch.
> This
> > >> > is similar to how we have checksums on regular data.  Or if the
> > >> > question is about catching logic errors in the metadata handling
> code,
> > >> > that sounds more like something that should be caught by test cases.
> > >> >
> > >> > best,
> > >> > Colin
> > >> >
> > >> >
> > >> > > Thanks!
> > >> > >
> > >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
> > >> wrote:
> > >> > >
> > >> > > > Hi Mickael,
> > >> > > >
> > >> > > > Thanks for taking a look.
> > >> > > >
> > >> > > > I don't think we want to support that kind of multi-tenancy at
> the
> > >> > > > controller level.  If the cluster is small enough that we want
> to
> > >> pack the
> > >> > > > controller(s) with something else, we could run them alongside
> the
> > >> brokers,
> > >> > > > or possibly inside three of the broker JVMs.
> > >> > > >
> > >> > > > best,
> > >> > > > Colin
> > >> > > >
> > >> > > >
> > >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > >> > > > > Thank Colin for kickstarting this initiative.
> > >> > > > >
> > >> > > > > Just one question.
> > >> > > > > - A nice feature of Zookeeper is the ability to use chroots
> and
> > >> have
> > >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is
> this
> > >> > > > > something we should keep?
> > >> > > > >
> > >> > > > > Thanks
> > >> > > > >
> > >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <
> cmccabe@apache.org>
> > >> wrote:
> > >> > > > > >
> > >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > >> > > > > > > Hi Colin,
> > >> > > > > > >
> > >> > > > > > > Thanks for the KIP.
> > >> > > > > > >
> > >> > > > > > > Currently ZooKeeper provides a convenient notification
> > >> mechanism for
> > >> > > > > > > knowing that broker and topic configuration has changed.
> While
> > >> > > > KIP-500 does
> > >> > > > > > > suggest that incremental metadata update is expected to
> come
> > >> to
> > >> > > > clients
> > >> > > > > > > eventually, that would seem to imply that for some number
> of
> > >> > > > releases there
> > >> > > > > > > would be no equivalent mechanism for knowing about config
> > >> changes.
> > >> > > > Is there
> > >> > > > > > > any thinking at this point about how a similar
> notification
> > >> might be
> > >> > > > > > > provided in the future?
> > >> > > > > >
> > >> > > > > > We could eventually have some inotify-like mechanism where
> > >> clients
> > >> > > > could register interest in various types of events and got
> notified
> > >> when
> > >> > > > they happened.  Reading the metadata log is conceptually simple.
> > >> The main
> > >> > > > complexity would be in setting up an API that made sense and
> that
> > >> didn't
> > >> > > > unduly constrain future implementations.  We'd have to think
> > >> carefully
> > >> > > > about what the real use-cases for this were, though.
> > >> > > > > >
> > >> > > > > > best,
> > >> > > > > > Colin
> > >> > > > > >
> > >> > > > > > >
> > >> > > > > > > Thanks,
> > >> > > > > > >
> > >> > > > > > > Tom
> > >> > > > > > >
> > >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > >> > > > viktorsomogyi@gmail.com>
> > >> > > > > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hey Colin,
> > >> > > > > > > >
> > >> > > > > > > > I think this is a long-awaited KIP, thanks for driving
> it.
> > >> I'm
> > >> > > > excited to
> > >> > > > > > > > see this in Kafka once. I collected my questions (and I
> > >> accept the
> > >> > > > "TBD"
> > >> > > > > > > > answer as they might be a bit deep for this high level
> :) ).
> > >> > > > > > > > 1.) Are there any specific reasons for the Controller
> just
> > >> > > > periodically
> > >> > > > > > > > persisting its state on disk periodically instead of
> > >> > > > asynchronously with
> > >> > > > > > > > every update? Wouldn't less frequent saves increase the
> > >> chance for
> > >> > > > missing
> > >> > > > > > > > a state change if the controller crashes between two
> saves?
> > >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from
> the
> > >> follower
> > >> > > > > > > > controllers? I assume that followers would have
> up-to-date
> > >> > > > information
> > >> > > > > > > > therefore brokers could fetch from there in theory.
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > > Viktor
> > >> > > > > > > >
> > >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > >> > > > reluctanthero104@gmail.com>
> > >> > > > > > > > wrote:
> > >> > > > > > > >
> > >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> > >> follow-up KIPs
> > >> > > > sounds
> > >> > > > > > > > like
> > >> > > > > > > > > a good idea.
> > >> > > > > > > > >
> > >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > >> ismael@juma.me.uk>
> > >> > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > > Hi Boyang,
> > >> > > > > > > > > >
> > >> > > > > > > > > > Yes, there will be several KIPs that will discuss
> the
> > >> items you
> > >> > > > > > > > describe
> > >> > > > > > > > > in
> > >> > > > > > > > > > detail. Colin, it may be helpful to make this clear
> in
> > >> the KIP
> > >> > > > 500
> > >> > > > > > > > > > description.
> > >> > > > > > > > > >
> > >> > > > > > > > > > Ismael
> > >> > > > > > > > > >
> > >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > >> > > > reluctanthero104@gmail.com
> > >> > > > > > > > >
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > > > Thanks Colin for initiating this important effort!
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > One question I have is whether we have a session
> > >> discussing
> > >> > > > the
> > >> > > > > > > > > > controller
> > >> > > > > > > > > > > failover in the new architecture? I know we are
> using
> > >> Raft
> > >> > > > protocol
> > >> > > > > > > > to
> > >> > > > > > > > > > > failover, yet it's still valuable to discuss the
> > >> steps new
> > >> > > > cluster is
> > >> > > > > > > > > > going
> > >> > > > > > > > > > > to take to reach the stable stage again, so that
> we
> > >> could
> > >> > > > easily
> > >> > > > > > > > > measure
> > >> > > > > > > > > > > the availability of the metadata servers.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Another suggestion I have is to write a
> step-by-step
> > >> design
> > >> > > > doc like
> > >> > > > > > > > > what
> > >> > > > > > > > > > > we did in KIP-98
> > >> > > > > > > > > > > <
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >> > > > > > > > > > > >,
> > >> > > > > > > > > > > including the new request protocols and how they
> are
> > >> > > > interacting in
> > >> > > > > > > > the
> > >> > > > > > > > > > new
> > >> > > > > > > > > > > cluster. For a complicated change like this, an
> > >> > > > implementation design
> > >> > > > > > > > > doc
> > >> > > > > > > > > > > help a lot in the review process, otherwise most
> > >> discussions
> > >> > > > we have
> > >> > > > > > > > > will
> > >> > > > > > > > > > > focus on high level and lose important details as
> we
> > >> > > > discover them in
> > >> > > > > > > > > the
> > >> > > > > > > > > > > post-agreement phase.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Boyang
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > >> > > > cmccabe@apache.org>
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando
> Garcia
> > >> Sancio
> > >> > > > wrote:
> > >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
> > >> comments
> > >> > > > and
> > >> > > > > > > > > questions.
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> > >> mentioned the
> > >> > > > > > > > LeaderAndIsr
> > >> > > > > > > > > > and
> > >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates
> which
> > >> the
> > >> > > > controller
> > >> > > > > > > > > > pushes,
> > >> > > > > > > > > > > > such
> > >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages".
> Is
> > >> your
> > >> > > > thinking
> > >> > > > > > > > that
> > >> > > > > > > > > > we
> > >> > > > > > > > > > > > will
> > >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > >> > > > UpdateMetadata only
> > >> > > > > > > > and
> > >> > > > > > > > > > add
> > >> > > > > > > > > > > > > topic configuration in this state?
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Hi Jose,
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Thanks for taking a look.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > The goal is for MetadataFetchRequest to replace
> both
> > >> > > > > > > > > > LeaderAndIsrRequest
> > >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
> > >> would be
> > >> > > > fetched
> > >> > > > > > > > > along
> > >> > > > > > > > > > > > with the other metadata.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > In the section "Broker Metadata Management",
> you
> > >> mention
> > >> > > > "Just
> > >> > > > > > > > like
> > >> > > > > > > > > > > with
> > >> > > > > > > > > > > > a
> > >> > > > > > > > > > > > > fetch request, the broker will track the
> offset
> > >> of the
> > >> > > > last
> > >> > > > > > > > updates
> > >> > > > > > > > > > it
> > >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> > >> requires that
> > >> > > > the
> > >> > > > > > > > > followers
> > >> > > > > > > > > > > > keep
> > >> > > > > > > > > > > > > all of the log entries (term/epoch and offset)
> > >> that are
> > >> > > > after the
> > >> > > > > > > > > > > > > highwatermark. Any log entry before the
> > >> highwatermark
> > >> > > > can be
> > >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> > >> MetadataFetch API
> > >> > > > to only
> > >> > > > > > > > > return
> > >> > > > > > > > > > > log
> > >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike the
> Raft
> > >> > > > replication API
> > >> > > > > > > > > > which
> > >> > > > > > > > > > > > > will replicate/fetch log entries after the
> > >> highwatermark
> > >> > > > for
> > >> > > > > > > > > > consensus?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> > >> metadata
> > >> > > > updates to
> > >> > > > > > > > the
> > >> > > > > > > > > > > > brokers until they've been stored on a majority
> of
> > >> the
> > >> > > > Raft nodes.
> > >> > > > > > > > > The
> > >> > > > > > > > > > > > most obvious way to do that, like you
> mentioned, is
> > >> to
> > >> > > > have the
> > >> > > > > > > > > brokers
> > >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
> > >> might be
> > >> > > > a more
> > >> > > > > > > > > clever
> > >> > > > > > > > > > > way
> > >> > > > > > > > > > > > to do it by fetching the data, but not having
> the
> > >> brokers
> > >> > > > act on it
> > >> > > > > > > > > > until
> > >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth
> it
> > >> or
> > >> > > > not.  We'll
> > >> > > > > > > > > > discuss
> > >> > > > > > > > > > > > this more in a separate KIP that just discusses
> > >> just Raft.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > >> mention "the
> > >> > > > > > > > > controller
> > >> > > > > > > > > > > will
> > >> > > > > > > > > > > > > send a full metadata image rather than a
> series of
> > >> > > > deltas". This
> > >> > > > > > > > > KIP
> > >> > > > > > > > > > > > > doesn't go into the set of operations that
> need
> > >> to be
> > >> > > > supported
> > >> > > > > > > > on
> > >> > > > > > > > > > top
> > >> > > > > > > > > > > of
> > >> > > > > > > > > > > > > Raft but it would be interested if this "full
> > >> metadata
> > >> > > > image"
> > >> > > > > > > > could
> > >> > > > > > > > > > be
> > >> > > > > > > > > > > > > express also as deltas. For example, assuming
> we
> > >> are
> > >> > > > replicating
> > >> > > > > > > > a
> > >> > > > > > > > > > map
> > >> > > > > > > > > > > > this
> > >> > > > > > > > > > > > > "full metadata image" could be a sequence of
> "put"
> > >> > > > operations
> > >> > > > > > > > > (znode
> > >> > > > > > > > > > > > create
> > >> > > > > > > > > > > > > to borrow ZK semantics).
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > The full image can definitely be expressed as a
> sum
> > >> of
> > >> > > > deltas.  At
> > >> > > > > > > > > some
> > >> > > > > > > > > > > > point, the number of deltas will get large
> enough
> > >> that
> > >> > > > sending a
> > >> > > > > > > > full
> > >> > > > > > > > > > > image
> > >> > > > > > > > > > > > is better, though.  One question that we're
> still
> > >> thinking
> > >> > > > about is
> > >> > > > > > > > > how
> > >> > > > > > > > > > > > much of this can be shared with generic Kafka
> log
> > >> code,
> > >> > > > and how
> > >> > > > > > > > much
> > >> > > > > > > > > > > should
> > >> > > > > > > > > > > > be different.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> > >> mention
> > >> > > > "This
> > >> > > > > > > > request
> > >> > > > > > > > > > will
> > >> > > > > > > > > > > > > double as a heartbeat, letting the controller
> > >> know that
> > >> > > > the
> > >> > > > > > > > broker
> > >> > > > > > > > > is
> > >> > > > > > > > > > > > > alive". In section "Broker State Machine", you
> > >> mention
> > >> > > > "The
> > >> > > > > > > > > > > MetadataFetch
> > >> > > > > > > > > > > > > API serves as this registration mechanism".
> Does
> > >> this
> > >> > > > mean that
> > >> > > > > > > > the
> > >> > > > > > > > > > > > > MetadataFetch Request will optionally include
> > >> broker
> > >> > > > > > > > configuration
> > >> > > > > > > > > > > > > information?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > I was originally thinking that the
> > >> MetadataFetchRequest
> > >> > > > should
> > >> > > > > > > > > include
> > >> > > > > > > > > > > > broker configuration information.  Thinking
> about
> > >> this
> > >> > > > more, maybe
> > >> > > > > > > > we
> > >> > > > > > > > > > > > should just have a special registration RPC that
> > >> contains
> > >> > > > that
> > >> > > > > > > > > > > information,
> > >> > > > > > > > > > > > to avoid sending it over the wire all the time.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > Does this also mean that MetadataFetch request
> > >> will
> > >> > > > result in
> > >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > >> replication
> > >> > > > protocol
> > >> > > > > > > > > before
> > >> > > > > > > > > > > you
> > >> > > > > > > > > > > > > can send the associated MetadataFetch
> Response?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > I think we should require the broker to be out
> of
> > >> the
> > >> > > > Offline state
> > >> > > > > > > > > > > before
> > >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the
> separate
> > >> > > > registration
> > >> > > > > > > > RPC
> > >> > > > > > > > > > > > should have completed first.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Broker State", you mention that a
> > >> broker can
> > >> > > > > > > > transition
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > > online after it is caught with the metadata.
> What
> > >> do you
> > >> > > > mean by
> > >> > > > > > > > > > this?
> > >> > > > > > > > > > > > > Metadata is always changing. How does the
> broker
> > >> know
> > >> > > > that it is
> > >> > > > > > > > > > caught
> > >> > > > > > > > > > > > up
> > >> > > > > > > > > > > > > since it doesn't participate in the consensus
> or
> > >> the
> > >> > > > advancement
> > >> > > > > > > > of
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > highwatermark?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > That's a good point.  Being "caught up" is
> somewhat
> > >> of a
> > >> > > > fuzzy
> > >> > > > > > > > > concept
> > >> > > > > > > > > > > > here, since the brokers do not participate in
> the
> > >> metadata
> > >> > > > > > > > consensus.
> > >> > > > > > > > > > I
> > >> > > > > > > > > > > > think ideally we would want to define it in
> terms
> > >> of time
> > >> > > > ("the
> > >> > > > > > > > > broker
> > >> > > > > > > > > > > has
> > >> > > > > > > > > > > > all the updates from the last 2 minutes", for
> > >> example.)
> > >> > > > We should
> > >> > > > > > > > > > spell
> > >> > > > > > > > > > > > this out better in the KIP.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > In section "Start the controller quorum
> nodes",
> > >> you
> > >> > > > mention "Once
> > >> > > > > > > > > it
> > >> > > > > > > > > > > has
> > >> > > > > > > > > > > > > taken over the /controller node, the active
> > >> controller
> > >> > > > will
> > >> > > > > > > > proceed
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > load
> > >> > > > > > > > > > > > > the full state of ZooKeeper.  It will write
> out
> > >> this
> > >> > > > information
> > >> > > > > > > > to
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > quorum's metadata storage.  After this point,
> the
> > >> > > > metadata quorum
> > >> > > > > > > > > > will
> > >> > > > > > > > > > > be
> > >> > > > > > > > > > > > > the metadata store of record, rather than the
> > >> data in
> > >> > > > ZooKeeper."
> > >> > > > > > > > > > > During
> > >> > > > > > > > > > > > > this migration do should we expect to have a
> > >> small period
> > >> > > > > > > > > controller
> > >> > > > > > > > > > > > > unavailability while the controller replicas
> this
> > >> state
> > >> > > > to all of
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > raft
> > >> > > > > > > > > > > > > nodes in the controller quorum and we buffer
> new
> > >> > > > controller API
> > >> > > > > > > > > > > requests?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Yes, the controller would be unavailable during
> this
> > >> > > > time.  I don't
> > >> > > > > > > > > > think
> > >> > > > > > > > > > > > this will be that different from the current
> period
> > >> of
> > >> > > > > > > > unavailability
> > >> > > > > > > > > > > when
> > >> > > > > > > > > > > > a new controller starts up and needs to load the
> > >> full
> > >> > > > state from
> > >> > > > > > > > ZK.
> > >> > > > > > > > > > The
> > >> > > > > > > > > > > > main difference is that in this period, we'd
> have
> > >> to write
> > >> > > > to the
> > >> > > > > > > > > > > > controller quorum rather than just to memory.
> But
> > >> we
> > >> > > > believe this
> > >> > > > > > > > > > should
> > >> > > > > > > > > > > > be pretty fast.
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > regards,
> > >> > > > > > > > > > > > Colin
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > Thanks!
> > >> > > > > > > > > > > > > -Jose
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> > >
> > >> > > --
> > >> > > David Arthur
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> Hi Colin.  I like the concept of a "bridge release" for migrating off of
> Zookeeper, but I worry that it may become a bottleneck if people hesitate
> to replace Zookeeper -- they would be unable to adopt newer versions of
> Kafka until taking (what feels to them like) a giant leap.  As an example,
> assuming version 4.0.x of Kafka is the supported bridge release, I  would
> not be surprised if uptake of the 4.x release and the time-based releases
> that follow it end up being much slower due to the perceived barrier.
> 
> Any perceived barrier could be lowered if the 4.0.x release could
> optionally continue to use Zookeeper -- then the cutover would be two
> incremental steps (move to 4.0.x, then replace Zookeeper while staying on
> 4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
> Zookeeper in one fell swoop).

Hi Ron,

Just to clarify, the "bridge release" will continue to use ZooKeeper.  It will not support running without ZooKeeper.  It is the releases that follow the bridge release that will remove ZooKeeper.

Also, it's a bit unclear whether the bridge release would be 3.x or 4.x, or something to follow.  We do know that the bridge release can't be a 2.x release, since it requires at least one incompatible change, removing --zookeeper options from all the shell scripts.  (Since we're doing semantic versioning, any time we make an incompatible change, we bump the major version number.)

In general, using two sources of metadata is a lot more complex and error-prone than one.  A lot of the bugs and corner cases we have are the result of divergences between the controller and the state in ZooKeeper.  Eliminating this divergence, and the split-brain scenarios it creates, is a major goal of this work.

> 
> Regardless of whether what I wrote above has merit or not, I think the KIP
> should be more explicit about what the upgrade constraints actually are.
> Can the bridge release be adopted with Zookeeper remaining in place and
> then cutting over as a second, follow-on step, or must the Controller
> Quorum nodes be started first and the bridge release cannot be used with
> Zookeeper at all?

As I mentioned above, the bridge release supports (indeed, requires) ZooKeeper.  I have added a little more text about this to KIP-500 which hopefully makes it clearer.

best,
Colin

>  If the bridge release cannot be used with Zookeeper at
> all, then no version at or beyond the bridge release is available
> unless/until abandoning Zookeeper; if the bridge release can be used with
> Zookeeper, then is it the only version that can be used with Zookeeper, or
> can Zookeeper be kept for additional releases if desired?
> 
> Ron
> 
> On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com> wrote:
> 
> > Hi Colin.  The diagram up at the top confused me -- specifically, the
> > lines connecting the controller/active-controller to the brokers.  I had
> > assumed the arrows on those lines represented the direction of data flow,
> > but that is not the case; the arrows actually identify the target of the
> > action, and the non-arrowed end indicates the initiator of the action.  For
> > example, the lines point from the controller to the brokers in the "today"
> > section on the left to show that the controller pushes to the brokers; the
> > lines point from the brokers to the active-controller in the "tomorrow"
> > section on the right to show that the brokers pull from the
> > active-controller.  As I said, this confused me because my gut instinct was
> > to interpret the arrow as indicating the direction of data flow, and when I
> > look at the "tomorrow" picture on the right I initially thought information
> > was moving from the brokers to the active-controller.  Did you consider
> > drawing that picture with the arrows reversed in the "tomorrow" side so
> > that the arrows represent the direction of data flow, and then add the
> > labels "push" on the "today" side and "pull" on the "tomorrow" side to
> > indicate who initiates the data flow?  It occurs to me that this picture
> > may end up being widely distributed, so it might be in everyone's interest
> > to proactively avoid any possible confusion by being more explicit.
> >
> > Minor corrections?
> > <<<In the current world, a broker which can contact ZooKeeper but which
> > is partitioned from the active controller
> > >>>In the current world, a broker which can contact ZooKeeper but which
> > is partitioned from the controller
> >
> > <<<Eventually, the controller will ask the broker to finally go offline
> > >>>Eventually, the active controller will ask the broker to finally go
> > offline
> >
> > <<<New versions of the clients should send these operations directly to
> > the controller
> > >>>New versions of the clients should send these operations directly to
> > the active controller
> >
> > <<<In the post-ZK world, the leader will make an RPC to the controller
> > instead
> > >>>In the post-ZK world, the leader will make an RPC to the active
> > controller instead
> >
> > <<<For example, the brokers may need to forward their requests to the
> > controller.
> > >>>For example, the brokers may need to forward their requests to the
> > active controller.
> >
> > <<<The new controller will monitor ZooKeeper for legacy broker node
> > registrations
> > >>>The new (active) controller will monitor ZooKeeper for legacy broker
> > node registrations
> >
> > Ron
> >
> > On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org> wrote:
> >
> >> Hi all,
> >>
> >> The KIP has been out for a while, so I'm thinking about calling a vote
> >> some time this week.
> >>
> >> best,
> >> Colin
> >>
> >> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> >> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> >> > > Thanks for the KIP, Colin. This looks great!
> >> > >
> >> > > I really like the idea of separating the Controller and Broker JVMs.
> >> > >
> >> > > As you alluded to above, it might be nice to have a separate
> >> > > broker-registration API to avoid overloading the metadata fetch API.
> >> > >
> >> >
> >> > Hi David,
> >> >
> >> > Thanks for taking a look.
> >> >
> >> > I removed the sentence about MetadataFetch also serving as the broker
> >> > registration API.  I think I agree that we will probably want a
> >> > separate RPC to fill this role.  We will have a follow-on KIP that will
> >> > go into more detail about metadata propagation and registration in the
> >> > post-ZK world.  That KIP will also have a full description of the
> >> > registration RPC, etc.  For now, I think the important part for KIP-500
> >> > is that the broker registers with the controller quorum.  On
> >> > registration, the controller quorum assigns it a new broker epoch,
> >> > which can distinguish successive broker incarnations.
> >> >
> >> > >
> >> > > When a broker gets a metadata delta, will it be a sequence of deltas
> >> since
> >> > > the last update or a cumulative delta since the last update?
> >> > >
> >> >
> >> > It will be a sequence of deltas.  Basically, the broker will be reading
> >> > from the metadata log.
> >> >
> >> > >
> >> > > Will we include any kind of integrity check on the deltas to ensure
> >> the brokers
> >> > > have applied them correctly? Perhaps this will be addressed in one of
> >> the
> >> > > follow-on KIPs.
> >> > >
> >> >
> >> > In general, we will have checksums on the metadata that we fetch.  This
> >> > is similar to how we have checksums on regular data.  Or if the
> >> > question is about catching logic errors in the metadata handling code,
> >> > that sounds more like something that should be caught by test cases.
> >> >
> >> > best,
> >> > Colin
> >> >
> >> >
> >> > > Thanks!
> >> > >
> >> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> > >
> >> > > > Hi Mickael,
> >> > > >
> >> > > > Thanks for taking a look.
> >> > > >
> >> > > > I don't think we want to support that kind of multi-tenancy at the
> >> > > > controller level.  If the cluster is small enough that we want to
> >> pack the
> >> > > > controller(s) with something else, we could run them alongside the
> >> brokers,
> >> > > > or possibly inside three of the broker JVMs.
> >> > > >
> >> > > > best,
> >> > > > Colin
> >> > > >
> >> > > >
> >> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> >> > > > > Thank Colin for kickstarting this initiative.
> >> > > > >
> >> > > > > Just one question.
> >> > > > > - A nice feature of Zookeeper is the ability to use chroots and
> >> have
> >> > > > > several Kafka clusters use the same Zookeeper ensemble. Is this
> >> > > > > something we should keep?
> >> > > > >
> >> > > > > Thanks
> >> > > > >
> >> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org>
> >> wrote:
> >> > > > > >
> >> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> >> > > > > > > Hi Colin,
> >> > > > > > >
> >> > > > > > > Thanks for the KIP.
> >> > > > > > >
> >> > > > > > > Currently ZooKeeper provides a convenient notification
> >> mechanism for
> >> > > > > > > knowing that broker and topic configuration has changed. While
> >> > > > KIP-500 does
> >> > > > > > > suggest that incremental metadata update is expected to come
> >> to
> >> > > > clients
> >> > > > > > > eventually, that would seem to imply that for some number of
> >> > > > releases there
> >> > > > > > > would be no equivalent mechanism for knowing about config
> >> changes.
> >> > > > Is there
> >> > > > > > > any thinking at this point about how a similar notification
> >> might be
> >> > > > > > > provided in the future?
> >> > > > > >
> >> > > > > > We could eventually have some inotify-like mechanism where
> >> clients
> >> > > > could register interest in various types of events and got notified
> >> when
> >> > > > they happened.  Reading the metadata log is conceptually simple.
> >> The main
> >> > > > complexity would be in setting up an API that made sense and that
> >> didn't
> >> > > > unduly constrain future implementations.  We'd have to think
> >> carefully
> >> > > > about what the real use-cases for this were, though.
> >> > > > > >
> >> > > > > > best,
> >> > > > > > Colin
> >> > > > > >
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > >
> >> > > > > > > Tom
> >> > > > > > >
> >> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> >> > > > viktorsomogyi@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hey Colin,
> >> > > > > > > >
> >> > > > > > > > I think this is a long-awaited KIP, thanks for driving it.
> >> I'm
> >> > > > excited to
> >> > > > > > > > see this in Kafka once. I collected my questions (and I
> >> accept the
> >> > > > "TBD"
> >> > > > > > > > answer as they might be a bit deep for this high level :) ).
> >> > > > > > > > 1.) Are there any specific reasons for the Controller just
> >> > > > periodically
> >> > > > > > > > persisting its state on disk periodically instead of
> >> > > > asynchronously with
> >> > > > > > > > every update? Wouldn't less frequent saves increase the
> >> chance for
> >> > > > missing
> >> > > > > > > > a state change if the controller crashes between two saves?
> >> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from the
> >> follower
> >> > > > > > > > controllers? I assume that followers would have up-to-date
> >> > > > information
> >> > > > > > > > therefore brokers could fetch from there in theory.
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > > Viktor
> >> > > > > > > >
> >> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> >> > > > reluctanthero104@gmail.com>
> >> > > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Thanks for explaining Ismael! Breaking down into
> >> follow-up KIPs
> >> > > > sounds
> >> > > > > > > > like
> >> > > > > > > > > a good idea.
> >> > > > > > > > >
> >> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> >> ismael@juma.me.uk>
> >> > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Hi Boyang,
> >> > > > > > > > > >
> >> > > > > > > > > > Yes, there will be several KIPs that will discuss the
> >> items you
> >> > > > > > > > describe
> >> > > > > > > > > in
> >> > > > > > > > > > detail. Colin, it may be helpful to make this clear in
> >> the KIP
> >> > > > 500
> >> > > > > > > > > > description.
> >> > > > > > > > > >
> >> > > > > > > > > > Ismael
> >> > > > > > > > > >
> >> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> >> > > > reluctanthero104@gmail.com
> >> > > > > > > > >
> >> > > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > Thanks Colin for initiating this important effort!
> >> > > > > > > > > > >
> >> > > > > > > > > > > One question I have is whether we have a session
> >> discussing
> >> > > > the
> >> > > > > > > > > > controller
> >> > > > > > > > > > > failover in the new architecture? I know we are using
> >> Raft
> >> > > > protocol
> >> > > > > > > > to
> >> > > > > > > > > > > failover, yet it's still valuable to discuss the
> >> steps new
> >> > > > cluster is
> >> > > > > > > > > > going
> >> > > > > > > > > > > to take to reach the stable stage again, so that we
> >> could
> >> > > > easily
> >> > > > > > > > > measure
> >> > > > > > > > > > > the availability of the metadata servers.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Another suggestion I have is to write a step-by-step
> >> design
> >> > > > doc like
> >> > > > > > > > > what
> >> > > > > > > > > > > we did in KIP-98
> >> > > > > > > > > > > <
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > > > > > > > > > >,
> >> > > > > > > > > > > including the new request protocols and how they are
> >> > > > interacting in
> >> > > > > > > > the
> >> > > > > > > > > > new
> >> > > > > > > > > > > cluster. For a complicated change like this, an
> >> > > > implementation design
> >> > > > > > > > > doc
> >> > > > > > > > > > > help a lot in the review process, otherwise most
> >> discussions
> >> > > > we have
> >> > > > > > > > > will
> >> > > > > > > > > > > focus on high level and lose important details as we
> >> > > > discover them in
> >> > > > > > > > > the
> >> > > > > > > > > > > post-agreement phase.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Boyang
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> >> > > > cmccabe@apache.org>
> >> > > > > > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia
> >> Sancio
> >> > > > wrote:
> >> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
> >> comments
> >> > > > and
> >> > > > > > > > > questions.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In the KIP's Motivation and Overview you
> >> mentioned the
> >> > > > > > > > LeaderAndIsr
> >> > > > > > > > > > and
> >> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates which
> >> the
> >> > > > controller
> >> > > > > > > > > > pushes,
> >> > > > > > > > > > > > such
> >> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is
> >> your
> >> > > > thinking
> >> > > > > > > > that
> >> > > > > > > > > > we
> >> > > > > > > > > > > > will
> >> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> >> > > > UpdateMetadata only
> >> > > > > > > > and
> >> > > > > > > > > > add
> >> > > > > > > > > > > > > topic configuration in this state?
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Hi Jose,
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Thanks for taking a look.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > The goal is for MetadataFetchRequest to replace both
> >> > > > > > > > > > LeaderAndIsrRequest
> >> > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
> >> would be
> >> > > > fetched
> >> > > > > > > > > along
> >> > > > > > > > > > > > with the other metadata.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > In the section "Broker Metadata Management", you
> >> mention
> >> > > > "Just
> >> > > > > > > > like
> >> > > > > > > > > > > with
> >> > > > > > > > > > > > a
> >> > > > > > > > > > > > > fetch request, the broker will track the offset
> >> of the
> >> > > > last
> >> > > > > > > > updates
> >> > > > > > > > > > it
> >> > > > > > > > > > > > > fetched". To keep the log consistent Raft
> >> requires that
> >> > > > the
> >> > > > > > > > > followers
> >> > > > > > > > > > > > keep
> >> > > > > > > > > > > > > all of the log entries (term/epoch and offset)
> >> that are
> >> > > > after the
> >> > > > > > > > > > > > > highwatermark. Any log entry before the
> >> highwatermark
> >> > > > can be
> >> > > > > > > > > > > > > compacted/snapshot. Do we expect the
> >> MetadataFetch API
> >> > > > to only
> >> > > > > > > > > return
> >> > > > > > > > > > > log
> >> > > > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> >> > > > replication API
> >> > > > > > > > > > which
> >> > > > > > > > > > > > > will replicate/fetch log entries after the
> >> highwatermark
> >> > > > for
> >> > > > > > > > > > consensus?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
> >> metadata
> >> > > > updates to
> >> > > > > > > > the
> >> > > > > > > > > > > > brokers until they've been stored on a majority of
> >> the
> >> > > > Raft nodes.
> >> > > > > > > > > The
> >> > > > > > > > > > > > most obvious way to do that, like you mentioned, is
> >> to
> >> > > > have the
> >> > > > > > > > > brokers
> >> > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
> >> might be
> >> > > > a more
> >> > > > > > > > > clever
> >> > > > > > > > > > > way
> >> > > > > > > > > > > > to do it by fetching the data, but not having the
> >> brokers
> >> > > > act on it
> >> > > > > > > > > > until
> >> > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth it
> >> or
> >> > > > not.  We'll
> >> > > > > > > > > > discuss
> >> > > > > > > > > > > > this more in a separate KIP that just discusses
> >> just Raft.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> >> mention "the
> >> > > > > > > > > controller
> >> > > > > > > > > > > will
> >> > > > > > > > > > > > > send a full metadata image rather than a series of
> >> > > > deltas". This
> >> > > > > > > > > KIP
> >> > > > > > > > > > > > > doesn't go into the set of operations that need
> >> to be
> >> > > > supported
> >> > > > > > > > on
> >> > > > > > > > > > top
> >> > > > > > > > > > > of
> >> > > > > > > > > > > > > Raft but it would be interested if this "full
> >> metadata
> >> > > > image"
> >> > > > > > > > could
> >> > > > > > > > > > be
> >> > > > > > > > > > > > > express also as deltas. For example, assuming we
> >> are
> >> > > > replicating
> >> > > > > > > > a
> >> > > > > > > > > > map
> >> > > > > > > > > > > > this
> >> > > > > > > > > > > > > "full metadata image" could be a sequence of "put"
> >> > > > operations
> >> > > > > > > > > (znode
> >> > > > > > > > > > > > create
> >> > > > > > > > > > > > > to borrow ZK semantics).
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > The full image can definitely be expressed as a sum
> >> of
> >> > > > deltas.  At
> >> > > > > > > > > some
> >> > > > > > > > > > > > point, the number of deltas will get large enough
> >> that
> >> > > > sending a
> >> > > > > > > > full
> >> > > > > > > > > > > image
> >> > > > > > > > > > > > is better, though.  One question that we're still
> >> thinking
> >> > > > about is
> >> > > > > > > > > how
> >> > > > > > > > > > > > much of this can be shared with generic Kafka log
> >> code,
> >> > > > and how
> >> > > > > > > > much
> >> > > > > > > > > > > should
> >> > > > > > > > > > > > be different.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Broker Metadata Management", you
> >> mention
> >> > > > "This
> >> > > > > > > > request
> >> > > > > > > > > > will
> >> > > > > > > > > > > > > double as a heartbeat, letting the controller
> >> know that
> >> > > > the
> >> > > > > > > > broker
> >> > > > > > > > > is
> >> > > > > > > > > > > > > alive". In section "Broker State Machine", you
> >> mention
> >> > > > "The
> >> > > > > > > > > > > MetadataFetch
> >> > > > > > > > > > > > > API serves as this registration mechanism". Does
> >> this
> >> > > > mean that
> >> > > > > > > > the
> >> > > > > > > > > > > > > MetadataFetch Request will optionally include
> >> broker
> >> > > > > > > > configuration
> >> > > > > > > > > > > > > information?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > I was originally thinking that the
> >> MetadataFetchRequest
> >> > > > should
> >> > > > > > > > > include
> >> > > > > > > > > > > > broker configuration information.  Thinking about
> >> this
> >> > > > more, maybe
> >> > > > > > > > we
> >> > > > > > > > > > > > should just have a special registration RPC that
> >> contains
> >> > > > that
> >> > > > > > > > > > > information,
> >> > > > > > > > > > > > to avoid sending it over the wire all the time.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > Does this also mean that MetadataFetch request
> >> will
> >> > > > result in
> >> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> >> replication
> >> > > > protocol
> >> > > > > > > > > before
> >> > > > > > > > > > > you
> >> > > > > > > > > > > > > can send the associated MetadataFetch Response?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > I think we should require the broker to be out of
> >> the
> >> > > > Offline state
> >> > > > > > > > > > > before
> >> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> >> > > > registration
> >> > > > > > > > RPC
> >> > > > > > > > > > > > should have completed first.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Broker State", you mention that a
> >> broker can
> >> > > > > > > > transition
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > online after it is caught with the metadata. What
> >> do you
> >> > > > mean by
> >> > > > > > > > > > this?
> >> > > > > > > > > > > > > Metadata is always changing. How does the broker
> >> know
> >> > > > that it is
> >> > > > > > > > > > caught
> >> > > > > > > > > > > > up
> >> > > > > > > > > > > > > since it doesn't participate in the consensus or
> >> the
> >> > > > advancement
> >> > > > > > > > of
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > highwatermark?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > That's a good point.  Being "caught up" is somewhat
> >> of a
> >> > > > fuzzy
> >> > > > > > > > > concept
> >> > > > > > > > > > > > here, since the brokers do not participate in the
> >> metadata
> >> > > > > > > > consensus.
> >> > > > > > > > > > I
> >> > > > > > > > > > > > think ideally we would want to define it in terms
> >> of time
> >> > > > ("the
> >> > > > > > > > > broker
> >> > > > > > > > > > > has
> >> > > > > > > > > > > > all the updates from the last 2 minutes", for
> >> example.)
> >> > > > We should
> >> > > > > > > > > > spell
> >> > > > > > > > > > > > this out better in the KIP.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > In section "Start the controller quorum nodes",
> >> you
> >> > > > mention "Once
> >> > > > > > > > > it
> >> > > > > > > > > > > has
> >> > > > > > > > > > > > > taken over the /controller node, the active
> >> controller
> >> > > > will
> >> > > > > > > > proceed
> >> > > > > > > > > > to
> >> > > > > > > > > > > > load
> >> > > > > > > > > > > > > the full state of ZooKeeper.  It will write out
> >> this
> >> > > > information
> >> > > > > > > > to
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > quorum's metadata storage.  After this point, the
> >> > > > metadata quorum
> >> > > > > > > > > > will
> >> > > > > > > > > > > be
> >> > > > > > > > > > > > > the metadata store of record, rather than the
> >> data in
> >> > > > ZooKeeper."
> >> > > > > > > > > > > During
> >> > > > > > > > > > > > > this migration do should we expect to have a
> >> small period
> >> > > > > > > > > controller
> >> > > > > > > > > > > > > unavailability while the controller replicas this
> >> state
> >> > > > to all of
> >> > > > > > > > > the
> >> > > > > > > > > > > > raft
> >> > > > > > > > > > > > > nodes in the controller quorum and we buffer new
> >> > > > controller API
> >> > > > > > > > > > > requests?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Yes, the controller would be unavailable during this
> >> > > > time.  I don't
> >> > > > > > > > > > think
> >> > > > > > > > > > > > this will be that different from the current period
> >> of
> >> > > > > > > > unavailability
> >> > > > > > > > > > > when
> >> > > > > > > > > > > > a new controller starts up and needs to load the
> >> full
> >> > > > state from
> >> > > > > > > > ZK.
> >> > > > > > > > > > The
> >> > > > > > > > > > > > main difference is that in this period, we'd have
> >> to write
> >> > > > to the
> >> > > > > > > > > > > > controller quorum rather than just to memory.  But
> >> we
> >> > > > believe this
> >> > > > > > > > > > should
> >> > > > > > > > > > > > be pretty fast.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > regards,
> >> > > > > > > > > > > > Colin
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Thanks!
> >> > > > > > > > > > > > > -Jose
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> > >
> >> > > --
> >> > > David Arthur
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ron Dagostino <rn...@gmail.com>.
Hi Colin.  I like the concept of a "bridge release" for migrating off of
Zookeeper, but I worry that it may become a bottleneck if people hesitate
to replace Zookeeper -- they would be unable to adopt newer versions of
Kafka until taking (what feels to them like) a giant leap.  As an example,
assuming version 4.0.x of Kafka is the supported bridge release, I  would
not be surprised if uptake of the 4.x release and the time-based releases
that follow it end up being much slower due to the perceived barrier.

Any perceived barrier could be lowered if the 4.0.x release could
optionally continue to use Zookeeper -- then the cutover would be two
incremental steps (move to 4.0.x, then replace Zookeeper while staying on
4.0.x) as opposed to a single big-bang (upgrade to 4.0.x and replace
Zookeeper in one fell swoop).

Regardless of whether what I wrote above has merit or not, I think the KIP
should be more explicit about what the upgrade constraints actually are.
Can the bridge release be adopted with Zookeeper remaining in place and
then cutting over as a second, follow-on step, or must the Controller
Quorum nodes be started first and the bridge release cannot be used with
Zookeeper at all?  If the bridge release cannot be used with Zookeeper at
all, then no version at or beyond the bridge release is available
unless/until abandoning Zookeeper; if the bridge release can be used with
Zookeeper, then is it the only version that can be used with Zookeeper, or
can Zookeeper be kept for additional releases if desired?

Ron

On Tue, Aug 20, 2019 at 10:19 AM Ron Dagostino <rn...@gmail.com> wrote:

> Hi Colin.  The diagram up at the top confused me -- specifically, the
> lines connecting the controller/active-controller to the brokers.  I had
> assumed the arrows on those lines represented the direction of data flow,
> but that is not the case; the arrows actually identify the target of the
> action, and the non-arrowed end indicates the initiator of the action.  For
> example, the lines point from the controller to the brokers in the "today"
> section on the left to show that the controller pushes to the brokers; the
> lines point from the brokers to the active-controller in the "tomorrow"
> section on the right to show that the brokers pull from the
> active-controller.  As I said, this confused me because my gut instinct was
> to interpret the arrow as indicating the direction of data flow, and when I
> look at the "tomorrow" picture on the right I initially thought information
> was moving from the brokers to the active-controller.  Did you consider
> drawing that picture with the arrows reversed in the "tomorrow" side so
> that the arrows represent the direction of data flow, and then add the
> labels "push" on the "today" side and "pull" on the "tomorrow" side to
> indicate who initiates the data flow?  It occurs to me that this picture
> may end up being widely distributed, so it might be in everyone's interest
> to proactively avoid any possible confusion by being more explicit.
>
> Minor corrections?
> <<<In the current world, a broker which can contact ZooKeeper but which
> is partitioned from the active controller
> >>>In the current world, a broker which can contact ZooKeeper but which
> is partitioned from the controller
>
> <<<Eventually, the controller will ask the broker to finally go offline
> >>>Eventually, the active controller will ask the broker to finally go
> offline
>
> <<<New versions of the clients should send these operations directly to
> the controller
> >>>New versions of the clients should send these operations directly to
> the active controller
>
> <<<In the post-ZK world, the leader will make an RPC to the controller
> instead
> >>>In the post-ZK world, the leader will make an RPC to the active
> controller instead
>
> <<<For example, the brokers may need to forward their requests to the
> controller.
> >>>For example, the brokers may need to forward their requests to the
> active controller.
>
> <<<The new controller will monitor ZooKeeper for legacy broker node
> registrations
> >>>The new (active) controller will monitor ZooKeeper for legacy broker
> node registrations
>
> Ron
>
> On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org> wrote:
>
>> Hi all,
>>
>> The KIP has been out for a while, so I'm thinking about calling a vote
>> some time this week.
>>
>> best,
>> Colin
>>
>> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
>> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
>> > > Thanks for the KIP, Colin. This looks great!
>> > >
>> > > I really like the idea of separating the Controller and Broker JVMs.
>> > >
>> > > As you alluded to above, it might be nice to have a separate
>> > > broker-registration API to avoid overloading the metadata fetch API.
>> > >
>> >
>> > Hi David,
>> >
>> > Thanks for taking a look.
>> >
>> > I removed the sentence about MetadataFetch also serving as the broker
>> > registration API.  I think I agree that we will probably want a
>> > separate RPC to fill this role.  We will have a follow-on KIP that will
>> > go into more detail about metadata propagation and registration in the
>> > post-ZK world.  That KIP will also have a full description of the
>> > registration RPC, etc.  For now, I think the important part for KIP-500
>> > is that the broker registers with the controller quorum.  On
>> > registration, the controller quorum assigns it a new broker epoch,
>> > which can distinguish successive broker incarnations.
>> >
>> > >
>> > > When a broker gets a metadata delta, will it be a sequence of deltas
>> since
>> > > the last update or a cumulative delta since the last update?
>> > >
>> >
>> > It will be a sequence of deltas.  Basically, the broker will be reading
>> > from the metadata log.
>> >
>> > >
>> > > Will we include any kind of integrity check on the deltas to ensure
>> the brokers
>> > > have applied them correctly? Perhaps this will be addressed in one of
>> the
>> > > follow-on KIPs.
>> > >
>> >
>> > In general, we will have checksums on the metadata that we fetch.  This
>> > is similar to how we have checksums on regular data.  Or if the
>> > question is about catching logic errors in the metadata handling code,
>> > that sounds more like something that should be caught by test cases.
>> >
>> > best,
>> > Colin
>> >
>> >
>> > > Thanks!
>> > >
>> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
>> wrote:
>> > >
>> > > > Hi Mickael,
>> > > >
>> > > > Thanks for taking a look.
>> > > >
>> > > > I don't think we want to support that kind of multi-tenancy at the
>> > > > controller level.  If the cluster is small enough that we want to
>> pack the
>> > > > controller(s) with something else, we could run them alongside the
>> brokers,
>> > > > or possibly inside three of the broker JVMs.
>> > > >
>> > > > best,
>> > > > Colin
>> > > >
>> > > >
>> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
>> > > > > Thank Colin for kickstarting this initiative.
>> > > > >
>> > > > > Just one question.
>> > > > > - A nice feature of Zookeeper is the ability to use chroots and
>> have
>> > > > > several Kafka clusters use the same Zookeeper ensemble. Is this
>> > > > > something we should keep?
>> > > > >
>> > > > > Thanks
>> > > > >
>> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org>
>> wrote:
>> > > > > >
>> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
>> > > > > > > Hi Colin,
>> > > > > > >
>> > > > > > > Thanks for the KIP.
>> > > > > > >
>> > > > > > > Currently ZooKeeper provides a convenient notification
>> mechanism for
>> > > > > > > knowing that broker and topic configuration has changed. While
>> > > > KIP-500 does
>> > > > > > > suggest that incremental metadata update is expected to come
>> to
>> > > > clients
>> > > > > > > eventually, that would seem to imply that for some number of
>> > > > releases there
>> > > > > > > would be no equivalent mechanism for knowing about config
>> changes.
>> > > > Is there
>> > > > > > > any thinking at this point about how a similar notification
>> might be
>> > > > > > > provided in the future?
>> > > > > >
>> > > > > > We could eventually have some inotify-like mechanism where
>> clients
>> > > > could register interest in various types of events and got notified
>> when
>> > > > they happened.  Reading the metadata log is conceptually simple.
>> The main
>> > > > complexity would be in setting up an API that made sense and that
>> didn't
>> > > > unduly constrain future implementations.  We'd have to think
>> carefully
>> > > > about what the real use-cases for this were, though.
>> > > > > >
>> > > > > > best,
>> > > > > > Colin
>> > > > > >
>> > > > > > >
>> > > > > > > Thanks,
>> > > > > > >
>> > > > > > > Tom
>> > > > > > >
>> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
>> > > > viktorsomogyi@gmail.com>
>> > > > > > > wrote:
>> > > > > > >
>> > > > > > > > Hey Colin,
>> > > > > > > >
>> > > > > > > > I think this is a long-awaited KIP, thanks for driving it.
>> I'm
>> > > > excited to
>> > > > > > > > see this in Kafka once. I collected my questions (and I
>> accept the
>> > > > "TBD"
>> > > > > > > > answer as they might be a bit deep for this high level :) ).
>> > > > > > > > 1.) Are there any specific reasons for the Controller just
>> > > > periodically
>> > > > > > > > persisting its state on disk periodically instead of
>> > > > asynchronously with
>> > > > > > > > every update? Wouldn't less frequent saves increase the
>> chance for
>> > > > missing
>> > > > > > > > a state change if the controller crashes between two saves?
>> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from the
>> follower
>> > > > > > > > controllers? I assume that followers would have up-to-date
>> > > > information
>> > > > > > > > therefore brokers could fetch from there in theory.
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > > Viktor
>> > > > > > > >
>> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
>> > > > reluctanthero104@gmail.com>
>> > > > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Thanks for explaining Ismael! Breaking down into
>> follow-up KIPs
>> > > > sounds
>> > > > > > > > like
>> > > > > > > > > a good idea.
>> > > > > > > > >
>> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
>> ismael@juma.me.uk>
>> > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > Hi Boyang,
>> > > > > > > > > >
>> > > > > > > > > > Yes, there will be several KIPs that will discuss the
>> items you
>> > > > > > > > describe
>> > > > > > > > > in
>> > > > > > > > > > detail. Colin, it may be helpful to make this clear in
>> the KIP
>> > > > 500
>> > > > > > > > > > description.
>> > > > > > > > > >
>> > > > > > > > > > Ismael
>> > > > > > > > > >
>> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
>> > > > reluctanthero104@gmail.com
>> > > > > > > > >
>> > > > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > Thanks Colin for initiating this important effort!
>> > > > > > > > > > >
>> > > > > > > > > > > One question I have is whether we have a session
>> discussing
>> > > > the
>> > > > > > > > > > controller
>> > > > > > > > > > > failover in the new architecture? I know we are using
>> Raft
>> > > > protocol
>> > > > > > > > to
>> > > > > > > > > > > failover, yet it's still valuable to discuss the
>> steps new
>> > > > cluster is
>> > > > > > > > > > going
>> > > > > > > > > > > to take to reach the stable stage again, so that we
>> could
>> > > > easily
>> > > > > > > > > measure
>> > > > > > > > > > > the availability of the metadata servers.
>> > > > > > > > > > >
>> > > > > > > > > > > Another suggestion I have is to write a step-by-step
>> design
>> > > > doc like
>> > > > > > > > > what
>> > > > > > > > > > > we did in KIP-98
>> > > > > > > > > > > <
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > > > > > > > > > > >,
>> > > > > > > > > > > including the new request protocols and how they are
>> > > > interacting in
>> > > > > > > > the
>> > > > > > > > > > new
>> > > > > > > > > > > cluster. For a complicated change like this, an
>> > > > implementation design
>> > > > > > > > > doc
>> > > > > > > > > > > help a lot in the review process, otherwise most
>> discussions
>> > > > we have
>> > > > > > > > > will
>> > > > > > > > > > > focus on high level and lose important details as we
>> > > > discover them in
>> > > > > > > > > the
>> > > > > > > > > > > post-agreement phase.
>> > > > > > > > > > >
>> > > > > > > > > > > Boyang
>> > > > > > > > > > >
>> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
>> > > > cmccabe@apache.org>
>> > > > > > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia
>> Sancio
>> > > > wrote:
>> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
>> comments
>> > > > and
>> > > > > > > > > questions.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > In the KIP's Motivation and Overview you
>> mentioned the
>> > > > > > > > LeaderAndIsr
>> > > > > > > > > > and
>> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates which
>> the
>> > > > controller
>> > > > > > > > > > pushes,
>> > > > > > > > > > > > such
>> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is
>> your
>> > > > thinking
>> > > > > > > > that
>> > > > > > > > > > we
>> > > > > > > > > > > > will
>> > > > > > > > > > > > > use MetadataFetch as a replacement to just
>> > > > UpdateMetadata only
>> > > > > > > > and
>> > > > > > > > > > add
>> > > > > > > > > > > > > topic configuration in this state?
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > > > Hi Jose,
>> > > > > > > > > > > >
>> > > > > > > > > > > > Thanks for taking a look.
>> > > > > > > > > > > >
>> > > > > > > > > > > > The goal is for MetadataFetchRequest to replace both
>> > > > > > > > > > LeaderAndIsrRequest
>> > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
>> would be
>> > > > fetched
>> > > > > > > > > along
>> > > > > > > > > > > > with the other metadata.
>> > > > > > > > > > > >
>> > > > > > > > > > > > > In the section "Broker Metadata Management", you
>> mention
>> > > > "Just
>> > > > > > > > like
>> > > > > > > > > > > with
>> > > > > > > > > > > > a
>> > > > > > > > > > > > > fetch request, the broker will track the offset
>> of the
>> > > > last
>> > > > > > > > updates
>> > > > > > > > > > it
>> > > > > > > > > > > > > fetched". To keep the log consistent Raft
>> requires that
>> > > > the
>> > > > > > > > > followers
>> > > > > > > > > > > > keep
>> > > > > > > > > > > > > all of the log entries (term/epoch and offset)
>> that are
>> > > > after the
>> > > > > > > > > > > > > highwatermark. Any log entry before the
>> highwatermark
>> > > > can be
>> > > > > > > > > > > > > compacted/snapshot. Do we expect the
>> MetadataFetch API
>> > > > to only
>> > > > > > > > > return
>> > > > > > > > > > > log
>> > > > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
>> > > > replication API
>> > > > > > > > > > which
>> > > > > > > > > > > > > will replicate/fetch log entries after the
>> highwatermark
>> > > > for
>> > > > > > > > > > consensus?
>> > > > > > > > > > > >
>> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose
>> metadata
>> > > > updates to
>> > > > > > > > the
>> > > > > > > > > > > > brokers until they've been stored on a majority of
>> the
>> > > > Raft nodes.
>> > > > > > > > > The
>> > > > > > > > > > > > most obvious way to do that, like you mentioned, is
>> to
>> > > > have the
>> > > > > > > > > brokers
>> > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
>> might be
>> > > > a more
>> > > > > > > > > clever
>> > > > > > > > > > > way
>> > > > > > > > > > > > to do it by fetching the data, but not having the
>> brokers
>> > > > act on it
>> > > > > > > > > > until
>> > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth it
>> or
>> > > > not.  We'll
>> > > > > > > > > > discuss
>> > > > > > > > > > > > this more in a separate KIP that just discusses
>> just Raft.
>> > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > In section "Broker Metadata Management", you
>> mention "the
>> > > > > > > > > controller
>> > > > > > > > > > > will
>> > > > > > > > > > > > > send a full metadata image rather than a series of
>> > > > deltas". This
>> > > > > > > > > KIP
>> > > > > > > > > > > > > doesn't go into the set of operations that need
>> to be
>> > > > supported
>> > > > > > > > on
>> > > > > > > > > > top
>> > > > > > > > > > > of
>> > > > > > > > > > > > > Raft but it would be interested if this "full
>> metadata
>> > > > image"
>> > > > > > > > could
>> > > > > > > > > > be
>> > > > > > > > > > > > > express also as deltas. For example, assuming we
>> are
>> > > > replicating
>> > > > > > > > a
>> > > > > > > > > > map
>> > > > > > > > > > > > this
>> > > > > > > > > > > > > "full metadata image" could be a sequence of "put"
>> > > > operations
>> > > > > > > > > (znode
>> > > > > > > > > > > > create
>> > > > > > > > > > > > > to borrow ZK semantics).
>> > > > > > > > > > > >
>> > > > > > > > > > > > The full image can definitely be expressed as a sum
>> of
>> > > > deltas.  At
>> > > > > > > > > some
>> > > > > > > > > > > > point, the number of deltas will get large enough
>> that
>> > > > sending a
>> > > > > > > > full
>> > > > > > > > > > > image
>> > > > > > > > > > > > is better, though.  One question that we're still
>> thinking
>> > > > about is
>> > > > > > > > > how
>> > > > > > > > > > > > much of this can be shared with generic Kafka log
>> code,
>> > > > and how
>> > > > > > > > much
>> > > > > > > > > > > should
>> > > > > > > > > > > > be different.
>> > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > In section "Broker Metadata Management", you
>> mention
>> > > > "This
>> > > > > > > > request
>> > > > > > > > > > will
>> > > > > > > > > > > > > double as a heartbeat, letting the controller
>> know that
>> > > > the
>> > > > > > > > broker
>> > > > > > > > > is
>> > > > > > > > > > > > > alive". In section "Broker State Machine", you
>> mention
>> > > > "The
>> > > > > > > > > > > MetadataFetch
>> > > > > > > > > > > > > API serves as this registration mechanism". Does
>> this
>> > > > mean that
>> > > > > > > > the
>> > > > > > > > > > > > > MetadataFetch Request will optionally include
>> broker
>> > > > > > > > configuration
>> > > > > > > > > > > > > information?
>> > > > > > > > > > > >
>> > > > > > > > > > > > I was originally thinking that the
>> MetadataFetchRequest
>> > > > should
>> > > > > > > > > include
>> > > > > > > > > > > > broker configuration information.  Thinking about
>> this
>> > > > more, maybe
>> > > > > > > > we
>> > > > > > > > > > > > should just have a special registration RPC that
>> contains
>> > > > that
>> > > > > > > > > > > information,
>> > > > > > > > > > > > to avoid sending it over the wire all the time.
>> > > > > > > > > > > >
>> > > > > > > > > > > > > Does this also mean that MetadataFetch request
>> will
>> > > > result in
>> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
>> replication
>> > > > protocol
>> > > > > > > > > before
>> > > > > > > > > > > you
>> > > > > > > > > > > > > can send the associated MetadataFetch Response?
>> > > > > > > > > > > >
>> > > > > > > > > > > > I think we should require the broker to be out of
>> the
>> > > > Offline state
>> > > > > > > > > > > before
>> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
>> > > > registration
>> > > > > > > > RPC
>> > > > > > > > > > > > should have completed first.
>> > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > In section "Broker State", you mention that a
>> broker can
>> > > > > > > > transition
>> > > > > > > > > > to
>> > > > > > > > > > > > > online after it is caught with the metadata. What
>> do you
>> > > > mean by
>> > > > > > > > > > this?
>> > > > > > > > > > > > > Metadata is always changing. How does the broker
>> know
>> > > > that it is
>> > > > > > > > > > caught
>> > > > > > > > > > > > up
>> > > > > > > > > > > > > since it doesn't participate in the consensus or
>> the
>> > > > advancement
>> > > > > > > > of
>> > > > > > > > > > the
>> > > > > > > > > > > > > highwatermark?
>> > > > > > > > > > > >
>> > > > > > > > > > > > That's a good point.  Being "caught up" is somewhat
>> of a
>> > > > fuzzy
>> > > > > > > > > concept
>> > > > > > > > > > > > here, since the brokers do not participate in the
>> metadata
>> > > > > > > > consensus.
>> > > > > > > > > > I
>> > > > > > > > > > > > think ideally we would want to define it in terms
>> of time
>> > > > ("the
>> > > > > > > > > broker
>> > > > > > > > > > > has
>> > > > > > > > > > > > all the updates from the last 2 minutes", for
>> example.)
>> > > > We should
>> > > > > > > > > > spell
>> > > > > > > > > > > > this out better in the KIP.
>> > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > In section "Start the controller quorum nodes",
>> you
>> > > > mention "Once
>> > > > > > > > > it
>> > > > > > > > > > > has
>> > > > > > > > > > > > > taken over the /controller node, the active
>> controller
>> > > > will
>> > > > > > > > proceed
>> > > > > > > > > > to
>> > > > > > > > > > > > load
>> > > > > > > > > > > > > the full state of ZooKeeper.  It will write out
>> this
>> > > > information
>> > > > > > > > to
>> > > > > > > > > > the
>> > > > > > > > > > > > > quorum's metadata storage.  After this point, the
>> > > > metadata quorum
>> > > > > > > > > > will
>> > > > > > > > > > > be
>> > > > > > > > > > > > > the metadata store of record, rather than the
>> data in
>> > > > ZooKeeper."
>> > > > > > > > > > > During
>> > > > > > > > > > > > > this migration do should we expect to have a
>> small period
>> > > > > > > > > controller
>> > > > > > > > > > > > > unavailability while the controller replicas this
>> state
>> > > > to all of
>> > > > > > > > > the
>> > > > > > > > > > > > raft
>> > > > > > > > > > > > > nodes in the controller quorum and we buffer new
>> > > > controller API
>> > > > > > > > > > > requests?
>> > > > > > > > > > > >
>> > > > > > > > > > > > Yes, the controller would be unavailable during this
>> > > > time.  I don't
>> > > > > > > > > > think
>> > > > > > > > > > > > this will be that different from the current period
>> of
>> > > > > > > > unavailability
>> > > > > > > > > > > when
>> > > > > > > > > > > > a new controller starts up and needs to load the
>> full
>> > > > state from
>> > > > > > > > ZK.
>> > > > > > > > > > The
>> > > > > > > > > > > > main difference is that in this period, we'd have
>> to write
>> > > > to the
>> > > > > > > > > > > > controller quorum rather than just to memory.  But
>> we
>> > > > believe this
>> > > > > > > > > > should
>> > > > > > > > > > > > be pretty fast.
>> > > > > > > > > > > >
>> > > > > > > > > > > > regards,
>> > > > > > > > > > > > Colin
>> > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Thanks!
>> > > > > > > > > > > > > -Jose
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > >
>> > > >
>> > >
>> > >
>> > > --
>> > > David Arthur
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Tue, Aug 20, 2019, at 07:19, Ron Dagostino wrote:
> Hi Colin.  The diagram up at the top confused me -- specifically, the lines
> connecting the controller/active-controller to the brokers.  I had assumed
> the arrows on those lines represented the direction of data flow, but that
> is not the case; the arrows actually identify the target of the action, and
> the non-arrowed end indicates the initiator of the action.  For example,
> the lines point from the controller to the brokers in the "today" section
> on the left to show that the controller pushes to the brokers; the lines
> point from the brokers to the active-controller in the "tomorrow" section
> on the right to show that the brokers pull from the active-controller.  As
> I said, this confused me because my gut instinct was to interpret the arrow
> as indicating the direction of data flow, and when I look at the "tomorrow"
> picture on the right I initially thought information was moving from the
> brokers to the active-controller.  Did you consider drawing that picture
> with the arrows reversed in the "tomorrow" side so that the arrows
> represent the direction of data flow, and then add the labels "push" on the
> "today" side and "pull" on the "tomorrow" side to indicate who initiates
> the data flow?  It occurs to me that this picture may end up being widely
> distributed, so it might be in everyone's interest to proactively avoid any
> possible confusion by being more explicit.

Hi Ron,

That's an interesting point.  I agree that in the second picture, the direction of data flow is opposite the direction in which the RPC goes.  The data flows from controller to broker, but the RPC is actually made by the broker to the controller.

I think very typical for arrows to represent the RPCs that are made, rather than the direction that information is flowing in.  For example, when diagramming a TCP handshake, the first arrow is typically drawn from the client to the server, even if the larger purpose of the connection is to fetch data from the server.  This is sort of a convention for diagrams like this.  I think reversing it would probably create more confusion than it would prevent, especially because with the arrows reversed on the second picture, it would be a lot less apparent how it differs from the first.

> 
> Minor corrections?
> <<<In the current world, a broker which can contact ZooKeeper but which is
> partitioned from the active controller
> >>>In the current world, a broker which can contact ZooKeeper but which is
> partitioned from the controller
> 
> <<<Eventually, the controller will ask the broker to finally go offline
> >>>Eventually, the active controller will ask the broker to finally go
> offline
> 
> <<<New versions of the clients should send these operations directly to the
> controller
> >>>New versions of the clients should send these operations directly to the
> active controller
> 
> <<<In the post-ZK world, the leader will make an RPC to the controller
> instead
> >>>In the post-ZK world, the leader will make an RPC to the active
> controller instead
> 
> <<<For example, the brokers may need to forward their requests to the
> controller.
> >>>For example, the brokers may need to forward their requests to the
> active controller.
> 
> <<<The new controller will monitor ZooKeeper for legacy broker node
> registrations
> >>>The new (active) controller will monitor ZooKeeper for legacy broker
> node registrations

Thanks.  I fixed the wording here as you suggested.

regards,
Colin


> 
> Ron
> 
> On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > Hi all,
> >
> > The KIP has been out for a while, so I'm thinking about calling a vote
> > some time this week.
> >
> > best,
> > Colin
> >
> > On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > > Thanks for the KIP, Colin. This looks great!
> > > >
> > > > I really like the idea of separating the Controller and Broker JVMs.
> > > >
> > > > As you alluded to above, it might be nice to have a separate
> > > > broker-registration API to avoid overloading the metadata fetch API.
> > > >
> > >
> > > Hi David,
> > >
> > > Thanks for taking a look.
> > >
> > > I removed the sentence about MetadataFetch also serving as the broker
> > > registration API.  I think I agree that we will probably want a
> > > separate RPC to fill this role.  We will have a follow-on KIP that will
> > > go into more detail about metadata propagation and registration in the
> > > post-ZK world.  That KIP will also have a full description of the
> > > registration RPC, etc.  For now, I think the important part for KIP-500
> > > is that the broker registers with the controller quorum.  On
> > > registration, the controller quorum assigns it a new broker epoch,
> > > which can distinguish successive broker incarnations.
> > >
> > > >
> > > > When a broker gets a metadata delta, will it be a sequence of deltas
> > since
> > > > the last update or a cumulative delta since the last update?
> > > >
> > >
> > > It will be a sequence of deltas.  Basically, the broker will be reading
> > > from the metadata log.
> > >
> > > >
> > > > Will we include any kind of integrity check on the deltas to ensure
> > the brokers
> > > > have applied them correctly? Perhaps this will be addressed in one of
> > the
> > > > follow-on KIPs.
> > > >
> > >
> > > In general, we will have checksums on the metadata that we fetch.  This
> > > is similar to how we have checksums on regular data.  Or if the
> > > question is about catching logic errors in the metadata handling code,
> > > that sounds more like something that should be caught by test cases.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > > Thanks!
> > > >
> > > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > >
> > > > > Hi Mickael,
> > > > >
> > > > > Thanks for taking a look.
> > > > >
> > > > > I don't think we want to support that kind of multi-tenancy at the
> > > > > controller level.  If the cluster is small enough that we want to
> > pack the
> > > > > controller(s) with something else, we could run them alongside the
> > brokers,
> > > > > or possibly inside three of the broker JVMs.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > > > Thank Colin for kickstarting this initiative.
> > > > > >
> > > > > > Just one question.
> > > > > > - A nice feature of Zookeeper is the ability to use chroots and
> > have
> > > > > > several Kafka clusters use the same Zookeeper ensemble. Is this
> > > > > > something we should keep?
> > > > > >
> > > > > > Thanks
> > > > > >
> > > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > > > > >
> > > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > > > > > Hi Colin,
> > > > > > > >
> > > > > > > > Thanks for the KIP.
> > > > > > > >
> > > > > > > > Currently ZooKeeper provides a convenient notification
> > mechanism for
> > > > > > > > knowing that broker and topic configuration has changed. While
> > > > > KIP-500 does
> > > > > > > > suggest that incremental metadata update is expected to come to
> > > > > clients
> > > > > > > > eventually, that would seem to imply that for some number of
> > > > > releases there
> > > > > > > > would be no equivalent mechanism for knowing about config
> > changes.
> > > > > Is there
> > > > > > > > any thinking at this point about how a similar notification
> > might be
> > > > > > > > provided in the future?
> > > > > > >
> > > > > > > We could eventually have some inotify-like mechanism where
> > clients
> > > > > could register interest in various types of events and got notified
> > when
> > > > > they happened.  Reading the metadata log is conceptually simple.
> > The main
> > > > > complexity would be in setting up an API that made sense and that
> > didn't
> > > > > unduly constrain future implementations.  We'd have to think
> > carefully
> > > > > about what the real use-cases for this were, though.
> > > > > > >
> > > > > > > best,
> > > > > > > Colin
> > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Tom
> > > > > > > >
> > > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > > > viktorsomogyi@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Colin,
> > > > > > > > >
> > > > > > > > > I think this is a long-awaited KIP, thanks for driving it.
> > I'm
> > > > > excited to
> > > > > > > > > see this in Kafka once. I collected my questions (and I
> > accept the
> > > > > "TBD"
> > > > > > > > > answer as they might be a bit deep for this high level :) ).
> > > > > > > > > 1.) Are there any specific reasons for the Controller just
> > > > > periodically
> > > > > > > > > persisting its state on disk periodically instead of
> > > > > asynchronously with
> > > > > > > > > every update? Wouldn't less frequent saves increase the
> > chance for
> > > > > missing
> > > > > > > > > a state change if the controller crashes between two saves?
> > > > > > > > > 2.) Why can't we allow brokers to fetch metadata from the
> > follower
> > > > > > > > > controllers? I assume that followers would have up-to-date
> > > > > information
> > > > > > > > > therefore brokers could fetch from there in theory.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Viktor
> > > > > > > > >
> > > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > > > reluctanthero104@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for explaining Ismael! Breaking down into follow-up
> > KIPs
> > > > > sounds
> > > > > > > > > like
> > > > > > > > > > a good idea.
> > > > > > > > > >
> > > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> > ismael@juma.me.uk>
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Boyang,
> > > > > > > > > > >
> > > > > > > > > > > Yes, there will be several KIPs that will discuss the
> > items you
> > > > > > > > > describe
> > > > > > > > > > in
> > > > > > > > > > > detail. Colin, it may be helpful to make this clear in
> > the KIP
> > > > > 500
> > > > > > > > > > > description.
> > > > > > > > > > >
> > > > > > > > > > > Ismael
> > > > > > > > > > >
> > > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > > > reluctanthero104@gmail.com
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks Colin for initiating this important effort!
> > > > > > > > > > > >
> > > > > > > > > > > > One question I have is whether we have a session
> > discussing
> > > > > the
> > > > > > > > > > > controller
> > > > > > > > > > > > failover in the new architecture? I know we are using
> > Raft
> > > > > protocol
> > > > > > > > > to
> > > > > > > > > > > > failover, yet it's still valuable to discuss the steps
> > new
> > > > > cluster is
> > > > > > > > > > > going
> > > > > > > > > > > > to take to reach the stable stage again, so that we
> > could
> > > > > easily
> > > > > > > > > > measure
> > > > > > > > > > > > the availability of the metadata servers.
> > > > > > > > > > > >
> > > > > > > > > > > > Another suggestion I have is to write a step-by-step
> > design
> > > > > doc like
> > > > > > > > > > what
> > > > > > > > > > > > we did in KIP-98
> > > > > > > > > > > > <
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > > > >,
> > > > > > > > > > > > including the new request protocols and how they are
> > > > > interacting in
> > > > > > > > > the
> > > > > > > > > > > new
> > > > > > > > > > > > cluster. For a complicated change like this, an
> > > > > implementation design
> > > > > > > > > > doc
> > > > > > > > > > > > help a lot in the review process, otherwise most
> > discussions
> > > > > we have
> > > > > > > > > > will
> > > > > > > > > > > > focus on high level and lose important details as we
> > > > > discover them in
> > > > > > > > > > the
> > > > > > > > > > > > post-agreement phase.
> > > > > > > > > > > >
> > > > > > > > > > > > Boyang
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > > > cmccabe@apache.org>
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia
> > Sancio
> > > > > wrote:
> > > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
> > comments
> > > > > and
> > > > > > > > > > questions.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > In the KIP's Motivation and Overview you mentioned
> > the
> > > > > > > > > LeaderAndIsr
> > > > > > > > > > > and
> > > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates which the
> > > > > controller
> > > > > > > > > > > pushes,
> > > > > > > > > > > > > such
> > > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is
> > your
> > > > > thinking
> > > > > > > > > that
> > > > > > > > > > > we
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > > > UpdateMetadata only
> > > > > > > > > and
> > > > > > > > > > > add
> > > > > > > > > > > > > > topic configuration in this state?
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi Jose,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for taking a look.
> > > > > > > > > > > > >
> > > > > > > > > > > > > The goal is for MetadataFetchRequest to replace both
> > > > > > > > > > > LeaderAndIsrRequest
> > > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
> > would be
> > > > > fetched
> > > > > > > > > > along
> > > > > > > > > > > > > with the other metadata.
> > > > > > > > > > > > >
> > > > > > > > > > > > > > In the section "Broker Metadata Management", you
> > mention
> > > > > "Just
> > > > > > > > > like
> > > > > > > > > > > > with
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > fetch request, the broker will track the offset of
> > the
> > > > > last
> > > > > > > > > updates
> > > > > > > > > > > it
> > > > > > > > > > > > > > fetched". To keep the log consistent Raft requires
> > that
> > > > > the
> > > > > > > > > > followers
> > > > > > > > > > > > > keep
> > > > > > > > > > > > > > all of the log entries (term/epoch and offset)
> > that are
> > > > > after the
> > > > > > > > > > > > > > highwatermark. Any log entry before the
> > highwatermark
> > > > > can be
> > > > > > > > > > > > > > compacted/snapshot. Do we expect the MetadataFetch
> > API
> > > > > to only
> > > > > > > > > > return
> > > > > > > > > > > > log
> > > > > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> > > > > replication API
> > > > > > > > > > > which
> > > > > > > > > > > > > > will replicate/fetch log entries after the
> > highwatermark
> > > > > for
> > > > > > > > > > > consensus?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Good question.  Clearly, we shouldn't expose metadata
> > > > > updates to
> > > > > > > > > the
> > > > > > > > > > > > > brokers until they've been stored on a majority of
> > the
> > > > > Raft nodes.
> > > > > > > > > > The
> > > > > > > > > > > > > most obvious way to do that, like you mentioned, is
> > to
> > > > > have the
> > > > > > > > > > brokers
> > > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
> > might be
> > > > > a more
> > > > > > > > > > clever
> > > > > > > > > > > > way
> > > > > > > > > > > > > to do it by fetching the data, but not having the
> > brokers
> > > > > act on it
> > > > > > > > > > > until
> > > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth it or
> > > > > not.  We'll
> > > > > > > > > > > discuss
> > > > > > > > > > > > > this more in a separate KIP that just discusses just
> > Raft.
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > In section "Broker Metadata Management", you
> > mention "the
> > > > > > > > > > controller
> > > > > > > > > > > > will
> > > > > > > > > > > > > > send a full metadata image rather than a series of
> > > > > deltas". This
> > > > > > > > > > KIP
> > > > > > > > > > > > > > doesn't go into the set of operations that need to
> > be
> > > > > supported
> > > > > > > > > on
> > > > > > > > > > > top
> > > > > > > > > > > > of
> > > > > > > > > > > > > > Raft but it would be interested if this "full
> > metadata
> > > > > image"
> > > > > > > > > could
> > > > > > > > > > > be
> > > > > > > > > > > > > > express also as deltas. For example, assuming we
> > are
> > > > > replicating
> > > > > > > > > a
> > > > > > > > > > > map
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > "full metadata image" could be a sequence of "put"
> > > > > operations
> > > > > > > > > > (znode
> > > > > > > > > > > > > create
> > > > > > > > > > > > > > to borrow ZK semantics).
> > > > > > > > > > > > >
> > > > > > > > > > > > > The full image can definitely be expressed as a sum
> > of
> > > > > deltas.  At
> > > > > > > > > > some
> > > > > > > > > > > > > point, the number of deltas will get large enough
> > that
> > > > > sending a
> > > > > > > > > full
> > > > > > > > > > > > image
> > > > > > > > > > > > > is better, though.  One question that we're still
> > thinking
> > > > > about is
> > > > > > > > > > how
> > > > > > > > > > > > > much of this can be shared with generic Kafka log
> > code,
> > > > > and how
> > > > > > > > > much
> > > > > > > > > > > > should
> > > > > > > > > > > > > be different.
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > In section "Broker Metadata Management", you
> > mention
> > > > > "This
> > > > > > > > > request
> > > > > > > > > > > will
> > > > > > > > > > > > > > double as a heartbeat, letting the controller know
> > that
> > > > > the
> > > > > > > > > broker
> > > > > > > > > > is
> > > > > > > > > > > > > > alive". In section "Broker State Machine", you
> > mention
> > > > > "The
> > > > > > > > > > > > MetadataFetch
> > > > > > > > > > > > > > API serves as this registration mechanism". Does
> > this
> > > > > mean that
> > > > > > > > > the
> > > > > > > > > > > > > > MetadataFetch Request will optionally include
> > broker
> > > > > > > > > configuration
> > > > > > > > > > > > > > information?
> > > > > > > > > > > > >
> > > > > > > > > > > > > I was originally thinking that the
> > MetadataFetchRequest
> > > > > should
> > > > > > > > > > include
> > > > > > > > > > > > > broker configuration information.  Thinking about
> > this
> > > > > more, maybe
> > > > > > > > > we
> > > > > > > > > > > > > should just have a special registration RPC that
> > contains
> > > > > that
> > > > > > > > > > > > information,
> > > > > > > > > > > > > to avoid sending it over the wire all the time.
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Does this also mean that MetadataFetch request will
> > > > > result in
> > > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> > replication
> > > > > protocol
> > > > > > > > > > before
> > > > > > > > > > > > you
> > > > > > > > > > > > > > can send the associated MetadataFetch Response?
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think we should require the broker to be out of the
> > > > > Offline state
> > > > > > > > > > > > before
> > > > > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> > > > > registration
> > > > > > > > > RPC
> > > > > > > > > > > > > should have completed first.
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > In section "Broker State", you mention that a
> > broker can
> > > > > > > > > transition
> > > > > > > > > > > to
> > > > > > > > > > > > > > online after it is caught with the metadata. What
> > do you
> > > > > mean by
> > > > > > > > > > > this?
> > > > > > > > > > > > > > Metadata is always changing. How does the broker
> > know
> > > > > that it is
> > > > > > > > > > > caught
> > > > > > > > > > > > > up
> > > > > > > > > > > > > > since it doesn't participate in the consensus or
> > the
> > > > > advancement
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > > highwatermark?
> > > > > > > > > > > > >
> > > > > > > > > > > > > That's a good point.  Being "caught up" is somewhat
> > of a
> > > > > fuzzy
> > > > > > > > > > concept
> > > > > > > > > > > > > here, since the brokers do not participate in the
> > metadata
> > > > > > > > > consensus.
> > > > > > > > > > > I
> > > > > > > > > > > > > think ideally we would want to define it in terms of
> > time
> > > > > ("the
> > > > > > > > > > broker
> > > > > > > > > > > > has
> > > > > > > > > > > > > all the updates from the last 2 minutes", for
> > example.)
> > > > > We should
> > > > > > > > > > > spell
> > > > > > > > > > > > > this out better in the KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > In section "Start the controller quorum nodes", you
> > > > > mention "Once
> > > > > > > > > > it
> > > > > > > > > > > > has
> > > > > > > > > > > > > > taken over the /controller node, the active
> > controller
> > > > > will
> > > > > > > > > proceed
> > > > > > > > > > > to
> > > > > > > > > > > > > load
> > > > > > > > > > > > > > the full state of ZooKeeper.  It will write out
> > this
> > > > > information
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > quorum's metadata storage.  After this point, the
> > > > > metadata quorum
> > > > > > > > > > > will
> > > > > > > > > > > > be
> > > > > > > > > > > > > > the metadata store of record, rather than the data
> > in
> > > > > ZooKeeper."
> > > > > > > > > > > > During
> > > > > > > > > > > > > > this migration do should we expect to have a small
> > period
> > > > > > > > > > controller
> > > > > > > > > > > > > > unavailability while the controller replicas this
> > state
> > > > > to all of
> > > > > > > > > > the
> > > > > > > > > > > > > raft
> > > > > > > > > > > > > > nodes in the controller quorum and we buffer new
> > > > > controller API
> > > > > > > > > > > > requests?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Yes, the controller would be unavailable during this
> > > > > time.  I don't
> > > > > > > > > > > think
> > > > > > > > > > > > > this will be that different from the current period
> > of
> > > > > > > > > unavailability
> > > > > > > > > > > > when
> > > > > > > > > > > > > a new controller starts up and needs to load the full
> > > > > state from
> > > > > > > > > ZK.
> > > > > > > > > > > The
> > > > > > > > > > > > > main difference is that in this period, we'd have to
> > write
> > > > > to the
> > > > > > > > > > > > > controller quorum rather than just to memory.  But we
> > > > > believe this
> > > > > > > > > > > should
> > > > > > > > > > > > > be pretty fast.
> > > > > > > > > > > > >
> > > > > > > > > > > > > regards,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks!
> > > > > > > > > > > > > > -Jose
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > David Arthur
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ron Dagostino <rn...@gmail.com>.
Hi Colin.  The diagram up at the top confused me -- specifically, the lines
connecting the controller/active-controller to the brokers.  I had assumed
the arrows on those lines represented the direction of data flow, but that
is not the case; the arrows actually identify the target of the action, and
the non-arrowed end indicates the initiator of the action.  For example,
the lines point from the controller to the brokers in the "today" section
on the left to show that the controller pushes to the brokers; the lines
point from the brokers to the active-controller in the "tomorrow" section
on the right to show that the brokers pull from the active-controller.  As
I said, this confused me because my gut instinct was to interpret the arrow
as indicating the direction of data flow, and when I look at the "tomorrow"
picture on the right I initially thought information was moving from the
brokers to the active-controller.  Did you consider drawing that picture
with the arrows reversed in the "tomorrow" side so that the arrows
represent the direction of data flow, and then add the labels "push" on the
"today" side and "pull" on the "tomorrow" side to indicate who initiates
the data flow?  It occurs to me that this picture may end up being widely
distributed, so it might be in everyone's interest to proactively avoid any
possible confusion by being more explicit.

Minor corrections?
<<<In the current world, a broker which can contact ZooKeeper but which is
partitioned from the active controller
>>>In the current world, a broker which can contact ZooKeeper but which is
partitioned from the controller

<<<Eventually, the controller will ask the broker to finally go offline
>>>Eventually, the active controller will ask the broker to finally go
offline

<<<New versions of the clients should send these operations directly to the
controller
>>>New versions of the clients should send these operations directly to the
active controller

<<<In the post-ZK world, the leader will make an RPC to the controller
instead
>>>In the post-ZK world, the leader will make an RPC to the active
controller instead

<<<For example, the brokers may need to forward their requests to the
controller.
>>>For example, the brokers may need to forward their requests to the
active controller.

<<<The new controller will monitor ZooKeeper for legacy broker node
registrations
>>>The new (active) controller will monitor ZooKeeper for legacy broker
node registrations

Ron

On Mon, Aug 19, 2019 at 6:53 PM Colin McCabe <cm...@apache.org> wrote:

> Hi all,
>
> The KIP has been out for a while, so I'm thinking about calling a vote
> some time this week.
>
> best,
> Colin
>
> On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> > On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > > Thanks for the KIP, Colin. This looks great!
> > >
> > > I really like the idea of separating the Controller and Broker JVMs.
> > >
> > > As you alluded to above, it might be nice to have a separate
> > > broker-registration API to avoid overloading the metadata fetch API.
> > >
> >
> > Hi David,
> >
> > Thanks for taking a look.
> >
> > I removed the sentence about MetadataFetch also serving as the broker
> > registration API.  I think I agree that we will probably want a
> > separate RPC to fill this role.  We will have a follow-on KIP that will
> > go into more detail about metadata propagation and registration in the
> > post-ZK world.  That KIP will also have a full description of the
> > registration RPC, etc.  For now, I think the important part for KIP-500
> > is that the broker registers with the controller quorum.  On
> > registration, the controller quorum assigns it a new broker epoch,
> > which can distinguish successive broker incarnations.
> >
> > >
> > > When a broker gets a metadata delta, will it be a sequence of deltas
> since
> > > the last update or a cumulative delta since the last update?
> > >
> >
> > It will be a sequence of deltas.  Basically, the broker will be reading
> > from the metadata log.
> >
> > >
> > > Will we include any kind of integrity check on the deltas to ensure
> the brokers
> > > have applied them correctly? Perhaps this will be addressed in one of
> the
> > > follow-on KIPs.
> > >
> >
> > In general, we will have checksums on the metadata that we fetch.  This
> > is similar to how we have checksums on regular data.  Or if the
> > question is about catching logic errors in the metadata handling code,
> > that sounds more like something that should be caught by test cases.
> >
> > best,
> > Colin
> >
> >
> > > Thanks!
> > >
> > > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org>
> wrote:
> > >
> > > > Hi Mickael,
> > > >
> > > > Thanks for taking a look.
> > > >
> > > > I don't think we want to support that kind of multi-tenancy at the
> > > > controller level.  If the cluster is small enough that we want to
> pack the
> > > > controller(s) with something else, we could run them alongside the
> brokers,
> > > > or possibly inside three of the broker JVMs.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > > Thank Colin for kickstarting this initiative.
> > > > >
> > > > > Just one question.
> > > > > - A nice feature of Zookeeper is the ability to use chroots and
> have
> > > > > several Kafka clusters use the same Zookeeper ensemble. Is this
> > > > > something we should keep?
> > > > >
> > > > > Thanks
> > > > >
> > > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org>
> wrote:
> > > > > >
> > > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > > > > Hi Colin,
> > > > > > >
> > > > > > > Thanks for the KIP.
> > > > > > >
> > > > > > > Currently ZooKeeper provides a convenient notification
> mechanism for
> > > > > > > knowing that broker and topic configuration has changed. While
> > > > KIP-500 does
> > > > > > > suggest that incremental metadata update is expected to come to
> > > > clients
> > > > > > > eventually, that would seem to imply that for some number of
> > > > releases there
> > > > > > > would be no equivalent mechanism for knowing about config
> changes.
> > > > Is there
> > > > > > > any thinking at this point about how a similar notification
> might be
> > > > > > > provided in the future?
> > > > > >
> > > > > > We could eventually have some inotify-like mechanism where
> clients
> > > > could register interest in various types of events and got notified
> when
> > > > they happened.  Reading the metadata log is conceptually simple.
> The main
> > > > complexity would be in setting up an API that made sense and that
> didn't
> > > > unduly constrain future implementations.  We'd have to think
> carefully
> > > > about what the real use-cases for this were, though.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Tom
> > > > > > >
> > > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > > viktorsomogyi@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hey Colin,
> > > > > > > >
> > > > > > > > I think this is a long-awaited KIP, thanks for driving it.
> I'm
> > > > excited to
> > > > > > > > see this in Kafka once. I collected my questions (and I
> accept the
> > > > "TBD"
> > > > > > > > answer as they might be a bit deep for this high level :) ).
> > > > > > > > 1.) Are there any specific reasons for the Controller just
> > > > periodically
> > > > > > > > persisting its state on disk periodically instead of
> > > > asynchronously with
> > > > > > > > every update? Wouldn't less frequent saves increase the
> chance for
> > > > missing
> > > > > > > > a state change if the controller crashes between two saves?
> > > > > > > > 2.) Why can't we allow brokers to fetch metadata from the
> follower
> > > > > > > > controllers? I assume that followers would have up-to-date
> > > > information
> > > > > > > > therefore brokers could fetch from there in theory.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Viktor
> > > > > > > >
> > > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > > reluctanthero104@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for explaining Ismael! Breaking down into follow-up
> KIPs
> > > > sounds
> > > > > > > > like
> > > > > > > > > a good idea.
> > > > > > > > >
> > > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <
> ismael@juma.me.uk>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Boyang,
> > > > > > > > > >
> > > > > > > > > > Yes, there will be several KIPs that will discuss the
> items you
> > > > > > > > describe
> > > > > > > > > in
> > > > > > > > > > detail. Colin, it may be helpful to make this clear in
> the KIP
> > > > 500
> > > > > > > > > > description.
> > > > > > > > > >
> > > > > > > > > > Ismael
> > > > > > > > > >
> > > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > > reluctanthero104@gmail.com
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks Colin for initiating this important effort!
> > > > > > > > > > >
> > > > > > > > > > > One question I have is whether we have a session
> discussing
> > > > the
> > > > > > > > > > controller
> > > > > > > > > > > failover in the new architecture? I know we are using
> Raft
> > > > protocol
> > > > > > > > to
> > > > > > > > > > > failover, yet it's still valuable to discuss the steps
> new
> > > > cluster is
> > > > > > > > > > going
> > > > > > > > > > > to take to reach the stable stage again, so that we
> could
> > > > easily
> > > > > > > > > measure
> > > > > > > > > > > the availability of the metadata servers.
> > > > > > > > > > >
> > > > > > > > > > > Another suggestion I have is to write a step-by-step
> design
> > > > doc like
> > > > > > > > > what
> > > > > > > > > > > we did in KIP-98
> > > > > > > > > > > <
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > > >,
> > > > > > > > > > > including the new request protocols and how they are
> > > > interacting in
> > > > > > > > the
> > > > > > > > > > new
> > > > > > > > > > > cluster. For a complicated change like this, an
> > > > implementation design
> > > > > > > > > doc
> > > > > > > > > > > help a lot in the review process, otherwise most
> discussions
> > > > we have
> > > > > > > > > will
> > > > > > > > > > > focus on high level and lose important details as we
> > > > discover them in
> > > > > > > > > the
> > > > > > > > > > > post-agreement phase.
> > > > > > > > > > >
> > > > > > > > > > > Boyang
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > > cmccabe@apache.org>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia
> Sancio
> > > > wrote:
> > > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few
> comments
> > > > and
> > > > > > > > > questions.
> > > > > > > > > > > > >
> > > > > > > > > > > > > In the KIP's Motivation and Overview you mentioned
> the
> > > > > > > > LeaderAndIsr
> > > > > > > > > > and
> > > > > > > > > > > > > UpdateMetadata RPC. For example, "updates which the
> > > > controller
> > > > > > > > > > pushes,
> > > > > > > > > > > > such
> > > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is
> your
> > > > thinking
> > > > > > > > that
> > > > > > > > > > we
> > > > > > > > > > > > will
> > > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > > UpdateMetadata only
> > > > > > > > and
> > > > > > > > > > add
> > > > > > > > > > > > > topic configuration in this state?
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Hi Jose,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for taking a look.
> > > > > > > > > > > >
> > > > > > > > > > > > The goal is for MetadataFetchRequest to replace both
> > > > > > > > > > LeaderAndIsrRequest
> > > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations
> would be
> > > > fetched
> > > > > > > > > along
> > > > > > > > > > > > with the other metadata.
> > > > > > > > > > > >
> > > > > > > > > > > > > In the section "Broker Metadata Management", you
> mention
> > > > "Just
> > > > > > > > like
> > > > > > > > > > > with
> > > > > > > > > > > > a
> > > > > > > > > > > > > fetch request, the broker will track the offset of
> the
> > > > last
> > > > > > > > updates
> > > > > > > > > > it
> > > > > > > > > > > > > fetched". To keep the log consistent Raft requires
> that
> > > > the
> > > > > > > > > followers
> > > > > > > > > > > > keep
> > > > > > > > > > > > > all of the log entries (term/epoch and offset)
> that are
> > > > after the
> > > > > > > > > > > > > highwatermark. Any log entry before the
> highwatermark
> > > > can be
> > > > > > > > > > > > > compacted/snapshot. Do we expect the MetadataFetch
> API
> > > > to only
> > > > > > > > > return
> > > > > > > > > > > log
> > > > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> > > > replication API
> > > > > > > > > > which
> > > > > > > > > > > > > will replicate/fetch log entries after the
> highwatermark
> > > > for
> > > > > > > > > > consensus?
> > > > > > > > > > > >
> > > > > > > > > > > > Good question.  Clearly, we shouldn't expose metadata
> > > > updates to
> > > > > > > > the
> > > > > > > > > > > > brokers until they've been stored on a majority of
> the
> > > > Raft nodes.
> > > > > > > > > The
> > > > > > > > > > > > most obvious way to do that, like you mentioned, is
> to
> > > > have the
> > > > > > > > > brokers
> > > > > > > > > > > > only fetch up to the HWM, but not beyond.  There
> might be
> > > > a more
> > > > > > > > > clever
> > > > > > > > > > > way
> > > > > > > > > > > > to do it by fetching the data, but not having the
> brokers
> > > > act on it
> > > > > > > > > > until
> > > > > > > > > > > > the HWM advances.  I'm not sure if that's worth it or
> > > > not.  We'll
> > > > > > > > > > discuss
> > > > > > > > > > > > this more in a separate KIP that just discusses just
> Raft.
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > In section "Broker Metadata Management", you
> mention "the
> > > > > > > > > controller
> > > > > > > > > > > will
> > > > > > > > > > > > > send a full metadata image rather than a series of
> > > > deltas". This
> > > > > > > > > KIP
> > > > > > > > > > > > > doesn't go into the set of operations that need to
> be
> > > > supported
> > > > > > > > on
> > > > > > > > > > top
> > > > > > > > > > > of
> > > > > > > > > > > > > Raft but it would be interested if this "full
> metadata
> > > > image"
> > > > > > > > could
> > > > > > > > > > be
> > > > > > > > > > > > > express also as deltas. For example, assuming we
> are
> > > > replicating
> > > > > > > > a
> > > > > > > > > > map
> > > > > > > > > > > > this
> > > > > > > > > > > > > "full metadata image" could be a sequence of "put"
> > > > operations
> > > > > > > > > (znode
> > > > > > > > > > > > create
> > > > > > > > > > > > > to borrow ZK semantics).
> > > > > > > > > > > >
> > > > > > > > > > > > The full image can definitely be expressed as a sum
> of
> > > > deltas.  At
> > > > > > > > > some
> > > > > > > > > > > > point, the number of deltas will get large enough
> that
> > > > sending a
> > > > > > > > full
> > > > > > > > > > > image
> > > > > > > > > > > > is better, though.  One question that we're still
> thinking
> > > > about is
> > > > > > > > > how
> > > > > > > > > > > > much of this can be shared with generic Kafka log
> code,
> > > > and how
> > > > > > > > much
> > > > > > > > > > > should
> > > > > > > > > > > > be different.
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > In section "Broker Metadata Management", you
> mention
> > > > "This
> > > > > > > > request
> > > > > > > > > > will
> > > > > > > > > > > > > double as a heartbeat, letting the controller know
> that
> > > > the
> > > > > > > > broker
> > > > > > > > > is
> > > > > > > > > > > > > alive". In section "Broker State Machine", you
> mention
> > > > "The
> > > > > > > > > > > MetadataFetch
> > > > > > > > > > > > > API serves as this registration mechanism". Does
> this
> > > > mean that
> > > > > > > > the
> > > > > > > > > > > > > MetadataFetch Request will optionally include
> broker
> > > > > > > > configuration
> > > > > > > > > > > > > information?
> > > > > > > > > > > >
> > > > > > > > > > > > I was originally thinking that the
> MetadataFetchRequest
> > > > should
> > > > > > > > > include
> > > > > > > > > > > > broker configuration information.  Thinking about
> this
> > > > more, maybe
> > > > > > > > we
> > > > > > > > > > > > should just have a special registration RPC that
> contains
> > > > that
> > > > > > > > > > > information,
> > > > > > > > > > > > to avoid sending it over the wire all the time.
> > > > > > > > > > > >
> > > > > > > > > > > > > Does this also mean that MetadataFetch request will
> > > > result in
> > > > > > > > > > > > > a "write"/AppendEntries through the Raft
> replication
> > > > protocol
> > > > > > > > > before
> > > > > > > > > > > you
> > > > > > > > > > > > > can send the associated MetadataFetch Response?
> > > > > > > > > > > >
> > > > > > > > > > > > I think we should require the broker to be out of the
> > > > Offline state
> > > > > > > > > > > before
> > > > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> > > > registration
> > > > > > > > RPC
> > > > > > > > > > > > should have completed first.
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > In section "Broker State", you mention that a
> broker can
> > > > > > > > transition
> > > > > > > > > > to
> > > > > > > > > > > > > online after it is caught with the metadata. What
> do you
> > > > mean by
> > > > > > > > > > this?
> > > > > > > > > > > > > Metadata is always changing. How does the broker
> know
> > > > that it is
> > > > > > > > > > caught
> > > > > > > > > > > > up
> > > > > > > > > > > > > since it doesn't participate in the consensus or
> the
> > > > advancement
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > highwatermark?
> > > > > > > > > > > >
> > > > > > > > > > > > That's a good point.  Being "caught up" is somewhat
> of a
> > > > fuzzy
> > > > > > > > > concept
> > > > > > > > > > > > here, since the brokers do not participate in the
> metadata
> > > > > > > > consensus.
> > > > > > > > > > I
> > > > > > > > > > > > think ideally we would want to define it in terms of
> time
> > > > ("the
> > > > > > > > > broker
> > > > > > > > > > > has
> > > > > > > > > > > > all the updates from the last 2 minutes", for
> example.)
> > > > We should
> > > > > > > > > > spell
> > > > > > > > > > > > this out better in the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > In section "Start the controller quorum nodes", you
> > > > mention "Once
> > > > > > > > > it
> > > > > > > > > > > has
> > > > > > > > > > > > > taken over the /controller node, the active
> controller
> > > > will
> > > > > > > > proceed
> > > > > > > > > > to
> > > > > > > > > > > > load
> > > > > > > > > > > > > the full state of ZooKeeper.  It will write out
> this
> > > > information
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > quorum's metadata storage.  After this point, the
> > > > metadata quorum
> > > > > > > > > > will
> > > > > > > > > > > be
> > > > > > > > > > > > > the metadata store of record, rather than the data
> in
> > > > ZooKeeper."
> > > > > > > > > > > During
> > > > > > > > > > > > > this migration do should we expect to have a small
> period
> > > > > > > > > controller
> > > > > > > > > > > > > unavailability while the controller replicas this
> state
> > > > to all of
> > > > > > > > > the
> > > > > > > > > > > > raft
> > > > > > > > > > > > > nodes in the controller quorum and we buffer new
> > > > controller API
> > > > > > > > > > > requests?
> > > > > > > > > > > >
> > > > > > > > > > > > Yes, the controller would be unavailable during this
> > > > time.  I don't
> > > > > > > > > > think
> > > > > > > > > > > > this will be that different from the current period
> of
> > > > > > > > unavailability
> > > > > > > > > > > when
> > > > > > > > > > > > a new controller starts up and needs to load the full
> > > > state from
> > > > > > > > ZK.
> > > > > > > > > > The
> > > > > > > > > > > > main difference is that in this period, we'd have to
> write
> > > > to the
> > > > > > > > > > > > controller quorum rather than just to memory.  But we
> > > > believe this
> > > > > > > > > > should
> > > > > > > > > > > > be pretty fast.
> > > > > > > > > > > >
> > > > > > > > > > > > regards,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks!
> > > > > > > > > > > > > -Jose
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> > >
> > > --
> > > David Arthur
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

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

The KIP has been out for a while, so I'm thinking about calling a vote some time this week.

best,
Colin

On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > Thanks for the KIP, Colin. This looks great!
> > 
> > I really like the idea of separating the Controller and Broker JVMs.
> > 
> > As you alluded to above, it might be nice to have a separate
> > broker-registration API to avoid overloading the metadata fetch API.
> >
> 
> Hi David,
> 
> Thanks for taking a look.
> 
> I removed the sentence about MetadataFetch also serving as the broker 
> registration API.  I think I agree that we will probably want a 
> separate RPC to fill this role.  We will have a follow-on KIP that will 
> go into more detail about metadata propagation and registration in the 
> post-ZK world.  That KIP will also have a full description of the 
> registration RPC, etc.  For now, I think the important part for KIP-500 
> is that the broker registers with the controller quorum.  On 
> registration, the controller quorum assigns it a new broker epoch, 
> which can distinguish successive broker incarnations.
> 
> > 
> > When a broker gets a metadata delta, will it be a sequence of deltas since
> > the last update or a cumulative delta since the last update?
> >
> 
> It will be a sequence of deltas.  Basically, the broker will be reading 
> from the metadata log.
> 
> >
> > Will we include any kind of integrity check on the deltas to ensure the brokers
> > have applied them correctly? Perhaps this will be addressed in one of the
> > follow-on KIPs.
> > 
> 
> In general, we will have checksums on the metadata that we fetch.  This 
> is similar to how we have checksums on regular data.  Or if the 
> question is about catching logic errors in the metadata handling code, 
> that sounds more like something that should be caught by test cases.
> 
> best,
> Colin
> 
> 
> > Thanks!
> > 
> > On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org> wrote:
> > 
> > > Hi Mickael,
> > >
> > > Thanks for taking a look.
> > >
> > > I don't think we want to support that kind of multi-tenancy at the
> > > controller level.  If the cluster is small enough that we want to pack the
> > > controller(s) with something else, we could run them alongside the brokers,
> > > or possibly inside three of the broker JVMs.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > > Thank Colin for kickstarting this initiative.
> > > >
> > > > Just one question.
> > > > - A nice feature of Zookeeper is the ability to use chroots and have
> > > > several Kafka clusters use the same Zookeeper ensemble. Is this
> > > > something we should keep?
> > > >
> > > > Thanks
> > > >
> > > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org> wrote:
> > > > >
> > > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > > > Hi Colin,
> > > > > >
> > > > > > Thanks for the KIP.
> > > > > >
> > > > > > Currently ZooKeeper provides a convenient notification mechanism for
> > > > > > knowing that broker and topic configuration has changed. While
> > > KIP-500 does
> > > > > > suggest that incremental metadata update is expected to come to
> > > clients
> > > > > > eventually, that would seem to imply that for some number of
> > > releases there
> > > > > > would be no equivalent mechanism for knowing about config changes.
> > > Is there
> > > > > > any thinking at this point about how a similar notification might be
> > > > > > provided in the future?
> > > > >
> > > > > We could eventually have some inotify-like mechanism where clients
> > > could register interest in various types of events and got notified when
> > > they happened.  Reading the metadata log is conceptually simple.  The main
> > > complexity would be in setting up an API that made sense and that didn't
> > > unduly constrain future implementations.  We'd have to think carefully
> > > about what the real use-cases for this were, though.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Tom
> > > > > >
> > > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > > viktorsomogyi@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hey Colin,
> > > > > > >
> > > > > > > I think this is a long-awaited KIP, thanks for driving it. I'm
> > > excited to
> > > > > > > see this in Kafka once. I collected my questions (and I accept the
> > > "TBD"
> > > > > > > answer as they might be a bit deep for this high level :) ).
> > > > > > > 1.) Are there any specific reasons for the Controller just
> > > periodically
> > > > > > > persisting its state on disk periodically instead of
> > > asynchronously with
> > > > > > > every update? Wouldn't less frequent saves increase the chance for
> > > missing
> > > > > > > a state change if the controller crashes between two saves?
> > > > > > > 2.) Why can't we allow brokers to fetch metadata from the follower
> > > > > > > controllers? I assume that followers would have up-to-date
> > > information
> > > > > > > therefore brokers could fetch from there in theory.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Viktor
> > > > > > >
> > > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > > reluctanthero104@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for explaining Ismael! Breaking down into follow-up KIPs
> > > sounds
> > > > > > > like
> > > > > > > > a good idea.
> > > > > > > >
> > > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi Boyang,
> > > > > > > > >
> > > > > > > > > Yes, there will be several KIPs that will discuss the items you
> > > > > > > describe
> > > > > > > > in
> > > > > > > > > detail. Colin, it may be helpful to make this clear in the KIP
> > > 500
> > > > > > > > > description.
> > > > > > > > >
> > > > > > > > > Ismael
> > > > > > > > >
> > > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > > reluctanthero104@gmail.com
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks Colin for initiating this important effort!
> > > > > > > > > >
> > > > > > > > > > One question I have is whether we have a session discussing
> > > the
> > > > > > > > > controller
> > > > > > > > > > failover in the new architecture? I know we are using Raft
> > > protocol
> > > > > > > to
> > > > > > > > > > failover, yet it's still valuable to discuss the steps new
> > > cluster is
> > > > > > > > > going
> > > > > > > > > > to take to reach the stable stage again, so that we could
> > > easily
> > > > > > > > measure
> > > > > > > > > > the availability of the metadata servers.
> > > > > > > > > >
> > > > > > > > > > Another suggestion I have is to write a step-by-step design
> > > doc like
> > > > > > > > what
> > > > > > > > > > we did in KIP-98
> > > > > > > > > > <
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > >,
> > > > > > > > > > including the new request protocols and how they are
> > > interacting in
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > cluster. For a complicated change like this, an
> > > implementation design
> > > > > > > > doc
> > > > > > > > > > help a lot in the review process, otherwise most discussions
> > > we have
> > > > > > > > will
> > > > > > > > > > focus on high level and lose important details as we
> > > discover them in
> > > > > > > > the
> > > > > > > > > > post-agreement phase.
> > > > > > > > > >
> > > > > > > > > > Boyang
> > > > > > > > > >
> > > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > > cmccabe@apache.org>
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio
> > > wrote:
> > > > > > > > > > > > Thanks Colin for the detail KIP. I have a few comments
> > > and
> > > > > > > > questions.
> > > > > > > > > > > >
> > > > > > > > > > > > In the KIP's Motivation and Overview you mentioned the
> > > > > > > LeaderAndIsr
> > > > > > > > > and
> > > > > > > > > > > > UpdateMetadata RPC. For example, "updates which the
> > > controller
> > > > > > > > > pushes,
> > > > > > > > > > > such
> > > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your
> > > thinking
> > > > > > > that
> > > > > > > > > we
> > > > > > > > > > > will
> > > > > > > > > > > > use MetadataFetch as a replacement to just
> > > UpdateMetadata only
> > > > > > > and
> > > > > > > > > add
> > > > > > > > > > > > topic configuration in this state?
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Hi Jose,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for taking a look.
> > > > > > > > > > >
> > > > > > > > > > > The goal is for MetadataFetchRequest to replace both
> > > > > > > > > LeaderAndIsrRequest
> > > > > > > > > > > and UpdateMetadataRequest.  Topic configurations would be
> > > fetched
> > > > > > > > along
> > > > > > > > > > > with the other metadata.
> > > > > > > > > > >
> > > > > > > > > > > > In the section "Broker Metadata Management", you mention
> > > "Just
> > > > > > > like
> > > > > > > > > > with
> > > > > > > > > > > a
> > > > > > > > > > > > fetch request, the broker will track the offset of the
> > > last
> > > > > > > updates
> > > > > > > > > it
> > > > > > > > > > > > fetched". To keep the log consistent Raft requires that
> > > the
> > > > > > > > followers
> > > > > > > > > > > keep
> > > > > > > > > > > > all of the log entries (term/epoch and offset) that are
> > > after the
> > > > > > > > > > > > highwatermark. Any log entry before the highwatermark
> > > can be
> > > > > > > > > > > > compacted/snapshot. Do we expect the MetadataFetch API
> > > to only
> > > > > > > > return
> > > > > > > > > > log
> > > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> > > replication API
> > > > > > > > > which
> > > > > > > > > > > > will replicate/fetch log entries after the highwatermark
> > > for
> > > > > > > > > consensus?
> > > > > > > > > > >
> > > > > > > > > > > Good question.  Clearly, we shouldn't expose metadata
> > > updates to
> > > > > > > the
> > > > > > > > > > > brokers until they've been stored on a majority of the
> > > Raft nodes.
> > > > > > > > The
> > > > > > > > > > > most obvious way to do that, like you mentioned, is to
> > > have the
> > > > > > > > brokers
> > > > > > > > > > > only fetch up to the HWM, but not beyond.  There might be
> > > a more
> > > > > > > > clever
> > > > > > > > > > way
> > > > > > > > > > > to do it by fetching the data, but not having the brokers
> > > act on it
> > > > > > > > > until
> > > > > > > > > > > the HWM advances.  I'm not sure if that's worth it or
> > > not.  We'll
> > > > > > > > > discuss
> > > > > > > > > > > this more in a separate KIP that just discusses just Raft.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > In section "Broker Metadata Management", you mention "the
> > > > > > > > controller
> > > > > > > > > > will
> > > > > > > > > > > > send a full metadata image rather than a series of
> > > deltas". This
> > > > > > > > KIP
> > > > > > > > > > > > doesn't go into the set of operations that need to be
> > > supported
> > > > > > > on
> > > > > > > > > top
> > > > > > > > > > of
> > > > > > > > > > > > Raft but it would be interested if this "full metadata
> > > image"
> > > > > > > could
> > > > > > > > > be
> > > > > > > > > > > > express also as deltas. For example, assuming we are
> > > replicating
> > > > > > > a
> > > > > > > > > map
> > > > > > > > > > > this
> > > > > > > > > > > > "full metadata image" could be a sequence of "put"
> > > operations
> > > > > > > > (znode
> > > > > > > > > > > create
> > > > > > > > > > > > to borrow ZK semantics).
> > > > > > > > > > >
> > > > > > > > > > > The full image can definitely be expressed as a sum of
> > > deltas.  At
> > > > > > > > some
> > > > > > > > > > > point, the number of deltas will get large enough that
> > > sending a
> > > > > > > full
> > > > > > > > > > image
> > > > > > > > > > > is better, though.  One question that we're still thinking
> > > about is
> > > > > > > > how
> > > > > > > > > > > much of this can be shared with generic Kafka log code,
> > > and how
> > > > > > > much
> > > > > > > > > > should
> > > > > > > > > > > be different.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > In section "Broker Metadata Management", you mention
> > > "This
> > > > > > > request
> > > > > > > > > will
> > > > > > > > > > > > double as a heartbeat, letting the controller know that
> > > the
> > > > > > > broker
> > > > > > > > is
> > > > > > > > > > > > alive". In section "Broker State Machine", you mention
> > > "The
> > > > > > > > > > MetadataFetch
> > > > > > > > > > > > API serves as this registration mechanism". Does this
> > > mean that
> > > > > > > the
> > > > > > > > > > > > MetadataFetch Request will optionally include broker
> > > > > > > configuration
> > > > > > > > > > > > information?
> > > > > > > > > > >
> > > > > > > > > > > I was originally thinking that the MetadataFetchRequest
> > > should
> > > > > > > > include
> > > > > > > > > > > broker configuration information.  Thinking about this
> > > more, maybe
> > > > > > > we
> > > > > > > > > > > should just have a special registration RPC that contains
> > > that
> > > > > > > > > > information,
> > > > > > > > > > > to avoid sending it over the wire all the time.
> > > > > > > > > > >
> > > > > > > > > > > > Does this also mean that MetadataFetch request will
> > > result in
> > > > > > > > > > > > a "write"/AppendEntries through the Raft replication
> > > protocol
> > > > > > > > before
> > > > > > > > > > you
> > > > > > > > > > > > can send the associated MetadataFetch Response?
> > > > > > > > > > >
> > > > > > > > > > > I think we should require the broker to be out of the
> > > Offline state
> > > > > > > > > > before
> > > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> > > registration
> > > > > > > RPC
> > > > > > > > > > > should have completed first.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > In section "Broker State", you mention that a broker can
> > > > > > > transition
> > > > > > > > > to
> > > > > > > > > > > > online after it is caught with the metadata. What do you
> > > mean by
> > > > > > > > > this?
> > > > > > > > > > > > Metadata is always changing. How does the broker know
> > > that it is
> > > > > > > > > caught
> > > > > > > > > > > up
> > > > > > > > > > > > since it doesn't participate in the consensus or the
> > > advancement
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > highwatermark?
> > > > > > > > > > >
> > > > > > > > > > > That's a good point.  Being "caught up" is somewhat of a
> > > fuzzy
> > > > > > > > concept
> > > > > > > > > > > here, since the brokers do not participate in the metadata
> > > > > > > consensus.
> > > > > > > > > I
> > > > > > > > > > > think ideally we would want to define it in terms of time
> > > ("the
> > > > > > > > broker
> > > > > > > > > > has
> > > > > > > > > > > all the updates from the last 2 minutes", for example.)
> > > We should
> > > > > > > > > spell
> > > > > > > > > > > this out better in the KIP.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > In section "Start the controller quorum nodes", you
> > > mention "Once
> > > > > > > > it
> > > > > > > > > > has
> > > > > > > > > > > > taken over the /controller node, the active controller
> > > will
> > > > > > > proceed
> > > > > > > > > to
> > > > > > > > > > > load
> > > > > > > > > > > > the full state of ZooKeeper.  It will write out this
> > > information
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > quorum's metadata storage.  After this point, the
> > > metadata quorum
> > > > > > > > > will
> > > > > > > > > > be
> > > > > > > > > > > > the metadata store of record, rather than the data in
> > > ZooKeeper."
> > > > > > > > > > During
> > > > > > > > > > > > this migration do should we expect to have a small period
> > > > > > > > controller
> > > > > > > > > > > > unavailability while the controller replicas this state
> > > to all of
> > > > > > > > the
> > > > > > > > > > > raft
> > > > > > > > > > > > nodes in the controller quorum and we buffer new
> > > controller API
> > > > > > > > > > requests?
> > > > > > > > > > >
> > > > > > > > > > > Yes, the controller would be unavailable during this
> > > time.  I don't
> > > > > > > > > think
> > > > > > > > > > > this will be that different from the current period of
> > > > > > > unavailability
> > > > > > > > > > when
> > > > > > > > > > > a new controller starts up and needs to load the full
> > > state from
> > > > > > > ZK.
> > > > > > > > > The
> > > > > > > > > > > main difference is that in this period, we'd have to write
> > > to the
> > > > > > > > > > > controller quorum rather than just to memory.  But we
> > > believe this
> > > > > > > > > should
> > > > > > > > > > > be pretty fast.
> > > > > > > > > > >
> > > > > > > > > > > regards,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks!
> > > > > > > > > > > > -Jose
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> > 
> > 
> > -- 
> > David Arthur
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> Thanks for the KIP, Colin. This looks great!
> 
> I really like the idea of separating the Controller and Broker JVMs.
> 
> As you alluded to above, it might be nice to have a separate
> broker-registration API to avoid overloading the metadata fetch API.
>

Hi David,

Thanks for taking a look.

I removed the sentence about MetadataFetch also serving as the broker registration API.  I think I agree that we will probably want a separate RPC to fill this role.  We will have a follow-on KIP that will go into more detail about metadata propagation and registration in the post-ZK world.  That KIP will also have a full description of the registration RPC, etc.  For now, I think the important part for KIP-500 is that the broker registers with the controller quorum.  On registration, the controller quorum assigns it a new broker epoch, which can distinguish successive broker incarnations.

> 
> When a broker gets a metadata delta, will it be a sequence of deltas since
> the last update or a cumulative delta since the last update?
>

It will be a sequence of deltas.  Basically, the broker will be reading from the metadata log.

>
> Will we include any kind of integrity check on the deltas to ensure the brokers
> have applied them correctly? Perhaps this will be addressed in one of the
> follow-on KIPs.
> 

In general, we will have checksums on the metadata that we fetch.  This is similar to how we have checksums on regular data.  Or if the question is about catching logic errors in the metadata handling code, that sounds more like something that should be caught by test cases.

best,
Colin


> Thanks!
> 
> On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > Hi Mickael,
> >
> > Thanks for taking a look.
> >
> > I don't think we want to support that kind of multi-tenancy at the
> > controller level.  If the cluster is small enough that we want to pack the
> > controller(s) with something else, we could run them alongside the brokers,
> > or possibly inside three of the broker JVMs.
> >
> > best,
> > Colin
> >
> >
> > On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > > Thank Colin for kickstarting this initiative.
> > >
> > > Just one question.
> > > - A nice feature of Zookeeper is the ability to use chroots and have
> > > several Kafka clusters use the same Zookeeper ensemble. Is this
> > > something we should keep?
> > >
> > > Thanks
> > >
> > > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org> wrote:
> > > >
> > > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > > Hi Colin,
> > > > >
> > > > > Thanks for the KIP.
> > > > >
> > > > > Currently ZooKeeper provides a convenient notification mechanism for
> > > > > knowing that broker and topic configuration has changed. While
> > KIP-500 does
> > > > > suggest that incremental metadata update is expected to come to
> > clients
> > > > > eventually, that would seem to imply that for some number of
> > releases there
> > > > > would be no equivalent mechanism for knowing about config changes.
> > Is there
> > > > > any thinking at this point about how a similar notification might be
> > > > > provided in the future?
> > > >
> > > > We could eventually have some inotify-like mechanism where clients
> > could register interest in various types of events and got notified when
> > they happened.  Reading the metadata log is conceptually simple.  The main
> > complexity would be in setting up an API that made sense and that didn't
> > unduly constrain future implementations.  We'd have to think carefully
> > about what the real use-cases for this were, though.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Tom
> > > > >
> > > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> > viktorsomogyi@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey Colin,
> > > > > >
> > > > > > I think this is a long-awaited KIP, thanks for driving it. I'm
> > excited to
> > > > > > see this in Kafka once. I collected my questions (and I accept the
> > "TBD"
> > > > > > answer as they might be a bit deep for this high level :) ).
> > > > > > 1.) Are there any specific reasons for the Controller just
> > periodically
> > > > > > persisting its state on disk periodically instead of
> > asynchronously with
> > > > > > every update? Wouldn't less frequent saves increase the chance for
> > missing
> > > > > > a state change if the controller crashes between two saves?
> > > > > > 2.) Why can't we allow brokers to fetch metadata from the follower
> > > > > > controllers? I assume that followers would have up-to-date
> > information
> > > > > > therefore brokers could fetch from there in theory.
> > > > > >
> > > > > > Thanks,
> > > > > > Viktor
> > > > > >
> > > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> > reluctanthero104@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Thanks for explaining Ismael! Breaking down into follow-up KIPs
> > sounds
> > > > > > like
> > > > > > > a good idea.
> > > > > > >
> > > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk>
> > wrote:
> > > > > > >
> > > > > > > > Hi Boyang,
> > > > > > > >
> > > > > > > > Yes, there will be several KIPs that will discuss the items you
> > > > > > describe
> > > > > > > in
> > > > > > > > detail. Colin, it may be helpful to make this clear in the KIP
> > 500
> > > > > > > > description.
> > > > > > > >
> > > > > > > > Ismael
> > > > > > > >
> > > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> > reluctanthero104@gmail.com
> > > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks Colin for initiating this important effort!
> > > > > > > > >
> > > > > > > > > One question I have is whether we have a session discussing
> > the
> > > > > > > > controller
> > > > > > > > > failover in the new architecture? I know we are using Raft
> > protocol
> > > > > > to
> > > > > > > > > failover, yet it's still valuable to discuss the steps new
> > cluster is
> > > > > > > > going
> > > > > > > > > to take to reach the stable stage again, so that we could
> > easily
> > > > > > > measure
> > > > > > > > > the availability of the metadata servers.
> > > > > > > > >
> > > > > > > > > Another suggestion I have is to write a step-by-step design
> > doc like
> > > > > > > what
> > > > > > > > > we did in KIP-98
> > > > > > > > > <
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > >,
> > > > > > > > > including the new request protocols and how they are
> > interacting in
> > > > > > the
> > > > > > > > new
> > > > > > > > > cluster. For a complicated change like this, an
> > implementation design
> > > > > > > doc
> > > > > > > > > help a lot in the review process, otherwise most discussions
> > we have
> > > > > > > will
> > > > > > > > > focus on high level and lose important details as we
> > discover them in
> > > > > > > the
> > > > > > > > > post-agreement phase.
> > > > > > > > >
> > > > > > > > > Boyang
> > > > > > > > >
> > > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> > cmccabe@apache.org>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio
> > wrote:
> > > > > > > > > > > Thanks Colin for the detail KIP. I have a few comments
> > and
> > > > > > > questions.
> > > > > > > > > > >
> > > > > > > > > > > In the KIP's Motivation and Overview you mentioned the
> > > > > > LeaderAndIsr
> > > > > > > > and
> > > > > > > > > > > UpdateMetadata RPC. For example, "updates which the
> > controller
> > > > > > > > pushes,
> > > > > > > > > > such
> > > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your
> > thinking
> > > > > > that
> > > > > > > > we
> > > > > > > > > > will
> > > > > > > > > > > use MetadataFetch as a replacement to just
> > UpdateMetadata only
> > > > > > and
> > > > > > > > add
> > > > > > > > > > > topic configuration in this state?
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Hi Jose,
> > > > > > > > > >
> > > > > > > > > > Thanks for taking a look.
> > > > > > > > > >
> > > > > > > > > > The goal is for MetadataFetchRequest to replace both
> > > > > > > > LeaderAndIsrRequest
> > > > > > > > > > and UpdateMetadataRequest.  Topic configurations would be
> > fetched
> > > > > > > along
> > > > > > > > > > with the other metadata.
> > > > > > > > > >
> > > > > > > > > > > In the section "Broker Metadata Management", you mention
> > "Just
> > > > > > like
> > > > > > > > > with
> > > > > > > > > > a
> > > > > > > > > > > fetch request, the broker will track the offset of the
> > last
> > > > > > updates
> > > > > > > > it
> > > > > > > > > > > fetched". To keep the log consistent Raft requires that
> > the
> > > > > > > followers
> > > > > > > > > > keep
> > > > > > > > > > > all of the log entries (term/epoch and offset) that are
> > after the
> > > > > > > > > > > highwatermark. Any log entry before the highwatermark
> > can be
> > > > > > > > > > > compacted/snapshot. Do we expect the MetadataFetch API
> > to only
> > > > > > > return
> > > > > > > > > log
> > > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> > replication API
> > > > > > > > which
> > > > > > > > > > > will replicate/fetch log entries after the highwatermark
> > for
> > > > > > > > consensus?
> > > > > > > > > >
> > > > > > > > > > Good question.  Clearly, we shouldn't expose metadata
> > updates to
> > > > > > the
> > > > > > > > > > brokers until they've been stored on a majority of the
> > Raft nodes.
> > > > > > > The
> > > > > > > > > > most obvious way to do that, like you mentioned, is to
> > have the
> > > > > > > brokers
> > > > > > > > > > only fetch up to the HWM, but not beyond.  There might be
> > a more
> > > > > > > clever
> > > > > > > > > way
> > > > > > > > > > to do it by fetching the data, but not having the brokers
> > act on it
> > > > > > > > until
> > > > > > > > > > the HWM advances.  I'm not sure if that's worth it or
> > not.  We'll
> > > > > > > > discuss
> > > > > > > > > > this more in a separate KIP that just discusses just Raft.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > In section "Broker Metadata Management", you mention "the
> > > > > > > controller
> > > > > > > > > will
> > > > > > > > > > > send a full metadata image rather than a series of
> > deltas". This
> > > > > > > KIP
> > > > > > > > > > > doesn't go into the set of operations that need to be
> > supported
> > > > > > on
> > > > > > > > top
> > > > > > > > > of
> > > > > > > > > > > Raft but it would be interested if this "full metadata
> > image"
> > > > > > could
> > > > > > > > be
> > > > > > > > > > > express also as deltas. For example, assuming we are
> > replicating
> > > > > > a
> > > > > > > > map
> > > > > > > > > > this
> > > > > > > > > > > "full metadata image" could be a sequence of "put"
> > operations
> > > > > > > (znode
> > > > > > > > > > create
> > > > > > > > > > > to borrow ZK semantics).
> > > > > > > > > >
> > > > > > > > > > The full image can definitely be expressed as a sum of
> > deltas.  At
> > > > > > > some
> > > > > > > > > > point, the number of deltas will get large enough that
> > sending a
> > > > > > full
> > > > > > > > > image
> > > > > > > > > > is better, though.  One question that we're still thinking
> > about is
> > > > > > > how
> > > > > > > > > > much of this can be shared with generic Kafka log code,
> > and how
> > > > > > much
> > > > > > > > > should
> > > > > > > > > > be different.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > In section "Broker Metadata Management", you mention
> > "This
> > > > > > request
> > > > > > > > will
> > > > > > > > > > > double as a heartbeat, letting the controller know that
> > the
> > > > > > broker
> > > > > > > is
> > > > > > > > > > > alive". In section "Broker State Machine", you mention
> > "The
> > > > > > > > > MetadataFetch
> > > > > > > > > > > API serves as this registration mechanism". Does this
> > mean that
> > > > > > the
> > > > > > > > > > > MetadataFetch Request will optionally include broker
> > > > > > configuration
> > > > > > > > > > > information?
> > > > > > > > > >
> > > > > > > > > > I was originally thinking that the MetadataFetchRequest
> > should
> > > > > > > include
> > > > > > > > > > broker configuration information.  Thinking about this
> > more, maybe
> > > > > > we
> > > > > > > > > > should just have a special registration RPC that contains
> > that
> > > > > > > > > information,
> > > > > > > > > > to avoid sending it over the wire all the time.
> > > > > > > > > >
> > > > > > > > > > > Does this also mean that MetadataFetch request will
> > result in
> > > > > > > > > > > a "write"/AppendEntries through the Raft replication
> > protocol
> > > > > > > before
> > > > > > > > > you
> > > > > > > > > > > can send the associated MetadataFetch Response?
> > > > > > > > > >
> > > > > > > > > > I think we should require the broker to be out of the
> > Offline state
> > > > > > > > > before
> > > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> > registration
> > > > > > RPC
> > > > > > > > > > should have completed first.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > In section "Broker State", you mention that a broker can
> > > > > > transition
> > > > > > > > to
> > > > > > > > > > > online after it is caught with the metadata. What do you
> > mean by
> > > > > > > > this?
> > > > > > > > > > > Metadata is always changing. How does the broker know
> > that it is
> > > > > > > > caught
> > > > > > > > > > up
> > > > > > > > > > > since it doesn't participate in the consensus or the
> > advancement
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > highwatermark?
> > > > > > > > > >
> > > > > > > > > > That's a good point.  Being "caught up" is somewhat of a
> > fuzzy
> > > > > > > concept
> > > > > > > > > > here, since the brokers do not participate in the metadata
> > > > > > consensus.
> > > > > > > > I
> > > > > > > > > > think ideally we would want to define it in terms of time
> > ("the
> > > > > > > broker
> > > > > > > > > has
> > > > > > > > > > all the updates from the last 2 minutes", for example.)
> > We should
> > > > > > > > spell
> > > > > > > > > > this out better in the KIP.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > In section "Start the controller quorum nodes", you
> > mention "Once
> > > > > > > it
> > > > > > > > > has
> > > > > > > > > > > taken over the /controller node, the active controller
> > will
> > > > > > proceed
> > > > > > > > to
> > > > > > > > > > load
> > > > > > > > > > > the full state of ZooKeeper.  It will write out this
> > information
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > quorum's metadata storage.  After this point, the
> > metadata quorum
> > > > > > > > will
> > > > > > > > > be
> > > > > > > > > > > the metadata store of record, rather than the data in
> > ZooKeeper."
> > > > > > > > > During
> > > > > > > > > > > this migration do should we expect to have a small period
> > > > > > > controller
> > > > > > > > > > > unavailability while the controller replicas this state
> > to all of
> > > > > > > the
> > > > > > > > > > raft
> > > > > > > > > > > nodes in the controller quorum and we buffer new
> > controller API
> > > > > > > > > requests?
> > > > > > > > > >
> > > > > > > > > > Yes, the controller would be unavailable during this
> > time.  I don't
> > > > > > > > think
> > > > > > > > > > this will be that different from the current period of
> > > > > > unavailability
> > > > > > > > > when
> > > > > > > > > > a new controller starts up and needs to load the full
> > state from
> > > > > > ZK.
> > > > > > > > The
> > > > > > > > > > main difference is that in this period, we'd have to write
> > to the
> > > > > > > > > > controller quorum rather than just to memory.  But we
> > believe this
> > > > > > > > should
> > > > > > > > > > be pretty fast.
> > > > > > > > > >
> > > > > > > > > > regards,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Thanks!
> > > > > > > > > > > -Jose
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
> 
> 
> -- 
> David Arthur
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by David Arthur <mu...@gmail.com>.
Thanks for the KIP, Colin. This looks great!

I really like the idea of separating the Controller and Broker JVMs.

As you alluded to above, it might be nice to have a separate
broker-registration API to avoid overloading the metadata fetch API.

When a broker gets a metadata delta, will it be a sequence of deltas since
the last update or a cumulative delta since the last update? Will we
include any kind of integrity check on the deltas to ensure the brokers
have applied them correctly? Perhaps this will be addressed in one of the
follow-on KIPs.

Thanks!

On Fri, Aug 9, 2019 at 1:17 PM Colin McCabe <cm...@apache.org> wrote:

> Hi Mickael,
>
> Thanks for taking a look.
>
> I don't think we want to support that kind of multi-tenancy at the
> controller level.  If the cluster is small enough that we want to pack the
> controller(s) with something else, we could run them alongside the brokers,
> or possibly inside three of the broker JVMs.
>
> best,
> Colin
>
>
> On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> > Thank Colin for kickstarting this initiative.
> >
> > Just one question.
> > - A nice feature of Zookeeper is the ability to use chroots and have
> > several Kafka clusters use the same Zookeeper ensemble. Is this
> > something we should keep?
> >
> > Thanks
> >
> > On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > > Hi Colin,
> > > >
> > > > Thanks for the KIP.
> > > >
> > > > Currently ZooKeeper provides a convenient notification mechanism for
> > > > knowing that broker and topic configuration has changed. While
> KIP-500 does
> > > > suggest that incremental metadata update is expected to come to
> clients
> > > > eventually, that would seem to imply that for some number of
> releases there
> > > > would be no equivalent mechanism for knowing about config changes.
> Is there
> > > > any thinking at this point about how a similar notification might be
> > > > provided in the future?
> > >
> > > We could eventually have some inotify-like mechanism where clients
> could register interest in various types of events and got notified when
> they happened.  Reading the metadata log is conceptually simple.  The main
> complexity would be in setting up an API that made sense and that didn't
> unduly constrain future implementations.  We'd have to think carefully
> about what the real use-cases for this were, though.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Thanks,
> > > >
> > > > Tom
> > > >
> > > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <
> viktorsomogyi@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey Colin,
> > > > >
> > > > > I think this is a long-awaited KIP, thanks for driving it. I'm
> excited to
> > > > > see this in Kafka once. I collected my questions (and I accept the
> "TBD"
> > > > > answer as they might be a bit deep for this high level :) ).
> > > > > 1.) Are there any specific reasons for the Controller just
> periodically
> > > > > persisting its state on disk periodically instead of
> asynchronously with
> > > > > every update? Wouldn't less frequent saves increase the chance for
> missing
> > > > > a state change if the controller crashes between two saves?
> > > > > 2.) Why can't we allow brokers to fetch metadata from the follower
> > > > > controllers? I assume that followers would have up-to-date
> information
> > > > > therefore brokers could fetch from there in theory.
> > > > >
> > > > > Thanks,
> > > > > Viktor
> > > > >
> > > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <
> reluctanthero104@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Thanks for explaining Ismael! Breaking down into follow-up KIPs
> sounds
> > > > > like
> > > > > > a good idea.
> > > > > >
> > > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk>
> wrote:
> > > > > >
> > > > > > > Hi Boyang,
> > > > > > >
> > > > > > > Yes, there will be several KIPs that will discuss the items you
> > > > > describe
> > > > > > in
> > > > > > > detail. Colin, it may be helpful to make this clear in the KIP
> 500
> > > > > > > description.
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <
> reluctanthero104@gmail.com
> > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks Colin for initiating this important effort!
> > > > > > > >
> > > > > > > > One question I have is whether we have a session discussing
> the
> > > > > > > controller
> > > > > > > > failover in the new architecture? I know we are using Raft
> protocol
> > > > > to
> > > > > > > > failover, yet it's still valuable to discuss the steps new
> cluster is
> > > > > > > going
> > > > > > > > to take to reach the stable stage again, so that we could
> easily
> > > > > > measure
> > > > > > > > the availability of the metadata servers.
> > > > > > > >
> > > > > > > > Another suggestion I have is to write a step-by-step design
> doc like
> > > > > > what
> > > > > > > > we did in KIP-98
> > > > > > > > <
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > >,
> > > > > > > > including the new request protocols and how they are
> interacting in
> > > > > the
> > > > > > > new
> > > > > > > > cluster. For a complicated change like this, an
> implementation design
> > > > > > doc
> > > > > > > > help a lot in the review process, otherwise most discussions
> we have
> > > > > > will
> > > > > > > > focus on high level and lose important details as we
> discover them in
> > > > > > the
> > > > > > > > post-agreement phase.
> > > > > > > >
> > > > > > > > Boyang
> > > > > > > >
> > > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <
> cmccabe@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio
> wrote:
> > > > > > > > > > Thanks Colin for the detail KIP. I have a few comments
> and
> > > > > > questions.
> > > > > > > > > >
> > > > > > > > > > In the KIP's Motivation and Overview you mentioned the
> > > > > LeaderAndIsr
> > > > > > > and
> > > > > > > > > > UpdateMetadata RPC. For example, "updates which the
> controller
> > > > > > > pushes,
> > > > > > > > > such
> > > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your
> thinking
> > > > > that
> > > > > > > we
> > > > > > > > > will
> > > > > > > > > > use MetadataFetch as a replacement to just
> UpdateMetadata only
> > > > > and
> > > > > > > add
> > > > > > > > > > topic configuration in this state?
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > Hi Jose,
> > > > > > > > >
> > > > > > > > > Thanks for taking a look.
> > > > > > > > >
> > > > > > > > > The goal is for MetadataFetchRequest to replace both
> > > > > > > LeaderAndIsrRequest
> > > > > > > > > and UpdateMetadataRequest.  Topic configurations would be
> fetched
> > > > > > along
> > > > > > > > > with the other metadata.
> > > > > > > > >
> > > > > > > > > > In the section "Broker Metadata Management", you mention
> "Just
> > > > > like
> > > > > > > > with
> > > > > > > > > a
> > > > > > > > > > fetch request, the broker will track the offset of the
> last
> > > > > updates
> > > > > > > it
> > > > > > > > > > fetched". To keep the log consistent Raft requires that
> the
> > > > > > followers
> > > > > > > > > keep
> > > > > > > > > > all of the log entries (term/epoch and offset) that are
> after the
> > > > > > > > > > highwatermark. Any log entry before the highwatermark
> can be
> > > > > > > > > > compacted/snapshot. Do we expect the MetadataFetch API
> to only
> > > > > > return
> > > > > > > > log
> > > > > > > > > > entries up to the highwatermark?  Unlike the Raft
> replication API
> > > > > > > which
> > > > > > > > > > will replicate/fetch log entries after the highwatermark
> for
> > > > > > > consensus?
> > > > > > > > >
> > > > > > > > > Good question.  Clearly, we shouldn't expose metadata
> updates to
> > > > > the
> > > > > > > > > brokers until they've been stored on a majority of the
> Raft nodes.
> > > > > > The
> > > > > > > > > most obvious way to do that, like you mentioned, is to
> have the
> > > > > > brokers
> > > > > > > > > only fetch up to the HWM, but not beyond.  There might be
> a more
> > > > > > clever
> > > > > > > > way
> > > > > > > > > to do it by fetching the data, but not having the brokers
> act on it
> > > > > > > until
> > > > > > > > > the HWM advances.  I'm not sure if that's worth it or
> not.  We'll
> > > > > > > discuss
> > > > > > > > > this more in a separate KIP that just discusses just Raft.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > In section "Broker Metadata Management", you mention "the
> > > > > > controller
> > > > > > > > will
> > > > > > > > > > send a full metadata image rather than a series of
> deltas". This
> > > > > > KIP
> > > > > > > > > > doesn't go into the set of operations that need to be
> supported
> > > > > on
> > > > > > > top
> > > > > > > > of
> > > > > > > > > > Raft but it would be interested if this "full metadata
> image"
> > > > > could
> > > > > > > be
> > > > > > > > > > express also as deltas. For example, assuming we are
> replicating
> > > > > a
> > > > > > > map
> > > > > > > > > this
> > > > > > > > > > "full metadata image" could be a sequence of "put"
> operations
> > > > > > (znode
> > > > > > > > > create
> > > > > > > > > > to borrow ZK semantics).
> > > > > > > > >
> > > > > > > > > The full image can definitely be expressed as a sum of
> deltas.  At
> > > > > > some
> > > > > > > > > point, the number of deltas will get large enough that
> sending a
> > > > > full
> > > > > > > > image
> > > > > > > > > is better, though.  One question that we're still thinking
> about is
> > > > > > how
> > > > > > > > > much of this can be shared with generic Kafka log code,
> and how
> > > > > much
> > > > > > > > should
> > > > > > > > > be different.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > In section "Broker Metadata Management", you mention
> "This
> > > > > request
> > > > > > > will
> > > > > > > > > > double as a heartbeat, letting the controller know that
> the
> > > > > broker
> > > > > > is
> > > > > > > > > > alive". In section "Broker State Machine", you mention
> "The
> > > > > > > > MetadataFetch
> > > > > > > > > > API serves as this registration mechanism". Does this
> mean that
> > > > > the
> > > > > > > > > > MetadataFetch Request will optionally include broker
> > > > > configuration
> > > > > > > > > > information?
> > > > > > > > >
> > > > > > > > > I was originally thinking that the MetadataFetchRequest
> should
> > > > > > include
> > > > > > > > > broker configuration information.  Thinking about this
> more, maybe
> > > > > we
> > > > > > > > > should just have a special registration RPC that contains
> that
> > > > > > > > information,
> > > > > > > > > to avoid sending it over the wire all the time.
> > > > > > > > >
> > > > > > > > > > Does this also mean that MetadataFetch request will
> result in
> > > > > > > > > > a "write"/AppendEntries through the Raft replication
> protocol
> > > > > > before
> > > > > > > > you
> > > > > > > > > > can send the associated MetadataFetch Response?
> > > > > > > > >
> > > > > > > > > I think we should require the broker to be out of the
> Offline state
> > > > > > > > before
> > > > > > > > > allowing it to fetch metadata, yes.  So the separate
> registration
> > > > > RPC
> > > > > > > > > should have completed first.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > In section "Broker State", you mention that a broker can
> > > > > transition
> > > > > > > to
> > > > > > > > > > online after it is caught with the metadata. What do you
> mean by
> > > > > > > this?
> > > > > > > > > > Metadata is always changing. How does the broker know
> that it is
> > > > > > > caught
> > > > > > > > > up
> > > > > > > > > > since it doesn't participate in the consensus or the
> advancement
> > > > > of
> > > > > > > the
> > > > > > > > > > highwatermark?
> > > > > > > > >
> > > > > > > > > That's a good point.  Being "caught up" is somewhat of a
> fuzzy
> > > > > > concept
> > > > > > > > > here, since the brokers do not participate in the metadata
> > > > > consensus.
> > > > > > > I
> > > > > > > > > think ideally we would want to define it in terms of time
> ("the
> > > > > > broker
> > > > > > > > has
> > > > > > > > > all the updates from the last 2 minutes", for example.)
> We should
> > > > > > > spell
> > > > > > > > > this out better in the KIP.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > In section "Start the controller quorum nodes", you
> mention "Once
> > > > > > it
> > > > > > > > has
> > > > > > > > > > taken over the /controller node, the active controller
> will
> > > > > proceed
> > > > > > > to
> > > > > > > > > load
> > > > > > > > > > the full state of ZooKeeper.  It will write out this
> information
> > > > > to
> > > > > > > the
> > > > > > > > > > quorum's metadata storage.  After this point, the
> metadata quorum
> > > > > > > will
> > > > > > > > be
> > > > > > > > > > the metadata store of record, rather than the data in
> ZooKeeper."
> > > > > > > > During
> > > > > > > > > > this migration do should we expect to have a small period
> > > > > > controller
> > > > > > > > > > unavailability while the controller replicas this state
> to all of
> > > > > > the
> > > > > > > > > raft
> > > > > > > > > > nodes in the controller quorum and we buffer new
> controller API
> > > > > > > > requests?
> > > > > > > > >
> > > > > > > > > Yes, the controller would be unavailable during this
> time.  I don't
> > > > > > > think
> > > > > > > > > this will be that different from the current period of
> > > > > unavailability
> > > > > > > > when
> > > > > > > > > a new controller starts up and needs to load the full
> state from
> > > > > ZK.
> > > > > > > The
> > > > > > > > > main difference is that in this period, we'd have to write
> to the
> > > > > > > > > controller quorum rather than just to memory.  But we
> believe this
> > > > > > > should
> > > > > > > > > be pretty fast.
> > > > > > > > >
> > > > > > > > > regards,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks!
> > > > > > > > > > -Jose
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
>


-- 
David Arthur

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

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

Thanks for taking a look.

I don't think we want to support that kind of multi-tenancy at the controller level.  If the cluster is small enough that we want to pack the controller(s) with something else, we could run them alongside the brokers, or possibly inside three of the broker JVMs.

best,
Colin


On Wed, Aug 7, 2019, at 10:37, Mickael Maison wrote:
> Thank Colin for kickstarting this initiative.
> 
> Just one question.
> - A nice feature of Zookeeper is the ability to use chroots and have
> several Kafka clusters use the same Zookeeper ensemble. Is this
> something we should keep?
> 
> Thanks
> 
> On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > > Hi Colin,
> > >
> > > Thanks for the KIP.
> > >
> > > Currently ZooKeeper provides a convenient notification mechanism for
> > > knowing that broker and topic configuration has changed. While KIP-500 does
> > > suggest that incremental metadata update is expected to come to clients
> > > eventually, that would seem to imply that for some number of releases there
> > > would be no equivalent mechanism for knowing about config changes. Is there
> > > any thinking at this point about how a similar notification might be
> > > provided in the future?
> >
> > We could eventually have some inotify-like mechanism where clients could register interest in various types of events and got notified when they happened.  Reading the metadata log is conceptually simple.  The main complexity would be in setting up an API that made sense and that didn't unduly constrain future implementations.  We'd have to think carefully about what the real use-cases for this were, though.
> >
> > best,
> > Colin
> >
> > >
> > > Thanks,
> > >
> > > Tom
> > >
> > > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <vi...@gmail.com>
> > > wrote:
> > >
> > > > Hey Colin,
> > > >
> > > > I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> > > > see this in Kafka once. I collected my questions (and I accept the "TBD"
> > > > answer as they might be a bit deep for this high level :) ).
> > > > 1.) Are there any specific reasons for the Controller just periodically
> > > > persisting its state on disk periodically instead of asynchronously with
> > > > every update? Wouldn't less frequent saves increase the chance for missing
> > > > a state change if the controller crashes between two saves?
> > > > 2.) Why can't we allow brokers to fetch metadata from the follower
> > > > controllers? I assume that followers would have up-to-date information
> > > > therefore brokers could fetch from there in theory.
> > > >
> > > > Thanks,
> > > > Viktor
> > > >
> > > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <re...@gmail.com>
> > > > wrote:
> > > >
> > > > > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds
> > > > like
> > > > > a good idea.
> > > > >
> > > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk> wrote:
> > > > >
> > > > > > Hi Boyang,
> > > > > >
> > > > > > Yes, there will be several KIPs that will discuss the items you
> > > > describe
> > > > > in
> > > > > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > > > > description.
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <reluctanthero104@gmail.com
> > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Thanks Colin for initiating this important effort!
> > > > > > >
> > > > > > > One question I have is whether we have a session discussing the
> > > > > > controller
> > > > > > > failover in the new architecture? I know we are using Raft protocol
> > > > to
> > > > > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > > > > going
> > > > > > > to take to reach the stable stage again, so that we could easily
> > > > > measure
> > > > > > > the availability of the metadata servers.
> > > > > > >
> > > > > > > Another suggestion I have is to write a step-by-step design doc like
> > > > > what
> > > > > > > we did in KIP-98
> > > > > > > <
> > > > > > >
> > > > > >
> > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > >,
> > > > > > > including the new request protocols and how they are interacting in
> > > > the
> > > > > > new
> > > > > > > cluster. For a complicated change like this, an implementation design
> > > > > doc
> > > > > > > help a lot in the review process, otherwise most discussions we have
> > > > > will
> > > > > > > focus on high level and lose important details as we discover them in
> > > > > the
> > > > > > > post-agreement phase.
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > > > > questions.
> > > > > > > > >
> > > > > > > > > In the KIP's Motivation and Overview you mentioned the
> > > > LeaderAndIsr
> > > > > > and
> > > > > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > > > > pushes,
> > > > > > > > such
> > > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking
> > > > that
> > > > > > we
> > > > > > > > will
> > > > > > > > > use MetadataFetch as a replacement to just UpdateMetadata only
> > > > and
> > > > > > add
> > > > > > > > > topic configuration in this state?
> > > > > > > > >
> > > > > > > >
> > > > > > > > Hi Jose,
> > > > > > > >
> > > > > > > > Thanks for taking a look.
> > > > > > > >
> > > > > > > > The goal is for MetadataFetchRequest to replace both
> > > > > > LeaderAndIsrRequest
> > > > > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > > > > along
> > > > > > > > with the other metadata.
> > > > > > > >
> > > > > > > > > In the section "Broker Metadata Management", you mention "Just
> > > > like
> > > > > > > with
> > > > > > > > a
> > > > > > > > > fetch request, the broker will track the offset of the last
> > > > updates
> > > > > > it
> > > > > > > > > fetched". To keep the log consistent Raft requires that the
> > > > > followers
> > > > > > > > keep
> > > > > > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> > > > > return
> > > > > > > log
> > > > > > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > > > > > which
> > > > > > > > > will replicate/fetch log entries after the highwatermark for
> > > > > > consensus?
> > > > > > > >
> > > > > > > > Good question.  Clearly, we shouldn't expose metadata updates to
> > > > the
> > > > > > > > brokers until they've been stored on a majority of the Raft nodes.
> > > > > The
> > > > > > > > most obvious way to do that, like you mentioned, is to have the
> > > > > brokers
> > > > > > > > only fetch up to the HWM, but not beyond.  There might be a more
> > > > > clever
> > > > > > > way
> > > > > > > > to do it by fetching the data, but not having the brokers act on it
> > > > > > until
> > > > > > > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> > > > > > discuss
> > > > > > > > this more in a separate KIP that just discusses just Raft.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > In section "Broker Metadata Management", you mention "the
> > > > > controller
> > > > > > > will
> > > > > > > > > send a full metadata image rather than a series of deltas". This
> > > > > KIP
> > > > > > > > > doesn't go into the set of operations that need to be supported
> > > > on
> > > > > > top
> > > > > > > of
> > > > > > > > > Raft but it would be interested if this "full metadata image"
> > > > could
> > > > > > be
> > > > > > > > > express also as deltas. For example, assuming we are replicating
> > > > a
> > > > > > map
> > > > > > > > this
> > > > > > > > > "full metadata image" could be a sequence of "put" operations
> > > > > (znode
> > > > > > > > create
> > > > > > > > > to borrow ZK semantics).
> > > > > > > >
> > > > > > > > The full image can definitely be expressed as a sum of deltas.  At
> > > > > some
> > > > > > > > point, the number of deltas will get large enough that sending a
> > > > full
> > > > > > > image
> > > > > > > > is better, though.  One question that we're still thinking about is
> > > > > how
> > > > > > > > much of this can be shared with generic Kafka log code, and how
> > > > much
> > > > > > > should
> > > > > > > > be different.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > In section "Broker Metadata Management", you mention "This
> > > > request
> > > > > > will
> > > > > > > > > double as a heartbeat, letting the controller know that the
> > > > broker
> > > > > is
> > > > > > > > > alive". In section "Broker State Machine", you mention "The
> > > > > > > MetadataFetch
> > > > > > > > > API serves as this registration mechanism". Does this mean that
> > > > the
> > > > > > > > > MetadataFetch Request will optionally include broker
> > > > configuration
> > > > > > > > > information?
> > > > > > > >
> > > > > > > > I was originally thinking that the MetadataFetchRequest should
> > > > > include
> > > > > > > > broker configuration information.  Thinking about this more, maybe
> > > > we
> > > > > > > > should just have a special registration RPC that contains that
> > > > > > > information,
> > > > > > > > to avoid sending it over the wire all the time.
> > > > > > > >
> > > > > > > > > Does this also mean that MetadataFetch request will result in
> > > > > > > > > a "write"/AppendEntries through the Raft replication protocol
> > > > > before
> > > > > > > you
> > > > > > > > > can send the associated MetadataFetch Response?
> > > > > > > >
> > > > > > > > I think we should require the broker to be out of the Offline state
> > > > > > > before
> > > > > > > > allowing it to fetch metadata, yes.  So the separate registration
> > > > RPC
> > > > > > > > should have completed first.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > In section "Broker State", you mention that a broker can
> > > > transition
> > > > > > to
> > > > > > > > > online after it is caught with the metadata. What do you mean by
> > > > > > this?
> > > > > > > > > Metadata is always changing. How does the broker know that it is
> > > > > > caught
> > > > > > > > up
> > > > > > > > > since it doesn't participate in the consensus or the advancement
> > > > of
> > > > > > the
> > > > > > > > > highwatermark?
> > > > > > > >
> > > > > > > > That's a good point.  Being "caught up" is somewhat of a fuzzy
> > > > > concept
> > > > > > > > here, since the brokers do not participate in the metadata
> > > > consensus.
> > > > > > I
> > > > > > > > think ideally we would want to define it in terms of time ("the
> > > > > broker
> > > > > > > has
> > > > > > > > all the updates from the last 2 minutes", for example.)  We should
> > > > > > spell
> > > > > > > > this out better in the KIP.
> > > > > > > >
> > > > > > > > >
> > > > > > > > > In section "Start the controller quorum nodes", you mention "Once
> > > > > it
> > > > > > > has
> > > > > > > > > taken over the /controller node, the active controller will
> > > > proceed
> > > > > > to
> > > > > > > > load
> > > > > > > > > the full state of ZooKeeper.  It will write out this information
> > > > to
> > > > > > the
> > > > > > > > > quorum's metadata storage.  After this point, the metadata quorum
> > > > > > will
> > > > > > > be
> > > > > > > > > the metadata store of record, rather than the data in ZooKeeper."
> > > > > > > During
> > > > > > > > > this migration do should we expect to have a small period
> > > > > controller
> > > > > > > > > unavailability while the controller replicas this state to all of
> > > > > the
> > > > > > > > raft
> > > > > > > > > nodes in the controller quorum and we buffer new controller API
> > > > > > > requests?
> > > > > > > >
> > > > > > > > Yes, the controller would be unavailable during this time.  I don't
> > > > > > think
> > > > > > > > this will be that different from the current period of
> > > > unavailability
> > > > > > > when
> > > > > > > > a new controller starts up and needs to load the full state from
> > > > ZK.
> > > > > > The
> > > > > > > > main difference is that in this period, we'd have to write to the
> > > > > > > > controller quorum rather than just to memory.  But we believe this
> > > > > > should
> > > > > > > > be pretty fast.
> > > > > > > >
> > > > > > > > regards,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks!
> > > > > > > > > -Jose
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Mickael Maison <mi...@gmail.com>.
Thank Colin for kickstarting this initiative.

Just one question.
- A nice feature of Zookeeper is the ability to use chroots and have
several Kafka clusters use the same Zookeeper ensemble. Is this
something we should keep?

Thanks

On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe <cm...@apache.org> wrote:
>
> On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> > Hi Colin,
> >
> > Thanks for the KIP.
> >
> > Currently ZooKeeper provides a convenient notification mechanism for
> > knowing that broker and topic configuration has changed. While KIP-500 does
> > suggest that incremental metadata update is expected to come to clients
> > eventually, that would seem to imply that for some number of releases there
> > would be no equivalent mechanism for knowing about config changes. Is there
> > any thinking at this point about how a similar notification might be
> > provided in the future?
>
> We could eventually have some inotify-like mechanism where clients could register interest in various types of events and got notified when they happened.  Reading the metadata log is conceptually simple.  The main complexity would be in setting up an API that made sense and that didn't unduly constrain future implementations.  We'd have to think carefully about what the real use-cases for this were, though.
>
> best,
> Colin
>
> >
> > Thanks,
> >
> > Tom
> >
> > On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <vi...@gmail.com>
> > wrote:
> >
> > > Hey Colin,
> > >
> > > I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> > > see this in Kafka once. I collected my questions (and I accept the "TBD"
> > > answer as they might be a bit deep for this high level :) ).
> > > 1.) Are there any specific reasons for the Controller just periodically
> > > persisting its state on disk periodically instead of asynchronously with
> > > every update? Wouldn't less frequent saves increase the chance for missing
> > > a state change if the controller crashes between two saves?
> > > 2.) Why can't we allow brokers to fetch metadata from the follower
> > > controllers? I assume that followers would have up-to-date information
> > > therefore brokers could fetch from there in theory.
> > >
> > > Thanks,
> > > Viktor
> > >
> > > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <re...@gmail.com>
> > > wrote:
> > >
> > > > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds
> > > like
> > > > a good idea.
> > > >
> > > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk> wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Yes, there will be several KIPs that will discuss the items you
> > > describe
> > > > in
> > > > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > > > description.
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <reluctanthero104@gmail.com
> > > >
> > > > > wrote:
> > > > >
> > > > > > Thanks Colin for initiating this important effort!
> > > > > >
> > > > > > One question I have is whether we have a session discussing the
> > > > > controller
> > > > > > failover in the new architecture? I know we are using Raft protocol
> > > to
> > > > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > > > going
> > > > > > to take to reach the stable stage again, so that we could easily
> > > > measure
> > > > > > the availability of the metadata servers.
> > > > > >
> > > > > > Another suggestion I have is to write a step-by-step design doc like
> > > > what
> > > > > > we did in KIP-98
> > > > > > <
> > > > > >
> > > > >
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > >,
> > > > > > including the new request protocols and how they are interacting in
> > > the
> > > > > new
> > > > > > cluster. For a complicated change like this, an implementation design
> > > > doc
> > > > > > help a lot in the review process, otherwise most discussions we have
> > > > will
> > > > > > focus on high level and lose important details as we discover them in
> > > > the
> > > > > > post-agreement phase.
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org>
> > > > wrote:
> > > > > >
> > > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > > > questions.
> > > > > > > >
> > > > > > > > In the KIP's Motivation and Overview you mentioned the
> > > LeaderAndIsr
> > > > > and
> > > > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > > > pushes,
> > > > > > > such
> > > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking
> > > that
> > > > > we
> > > > > > > will
> > > > > > > > use MetadataFetch as a replacement to just UpdateMetadata only
> > > and
> > > > > add
> > > > > > > > topic configuration in this state?
> > > > > > > >
> > > > > > >
> > > > > > > Hi Jose,
> > > > > > >
> > > > > > > Thanks for taking a look.
> > > > > > >
> > > > > > > The goal is for MetadataFetchRequest to replace both
> > > > > LeaderAndIsrRequest
> > > > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > > > along
> > > > > > > with the other metadata.
> > > > > > >
> > > > > > > > In the section "Broker Metadata Management", you mention "Just
> > > like
> > > > > > with
> > > > > > > a
> > > > > > > > fetch request, the broker will track the offset of the last
> > > updates
> > > > > it
> > > > > > > > fetched". To keep the log consistent Raft requires that the
> > > > followers
> > > > > > > keep
> > > > > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> > > > return
> > > > > > log
> > > > > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > > > > which
> > > > > > > > will replicate/fetch log entries after the highwatermark for
> > > > > consensus?
> > > > > > >
> > > > > > > Good question.  Clearly, we shouldn't expose metadata updates to
> > > the
> > > > > > > brokers until they've been stored on a majority of the Raft nodes.
> > > > The
> > > > > > > most obvious way to do that, like you mentioned, is to have the
> > > > brokers
> > > > > > > only fetch up to the HWM, but not beyond.  There might be a more
> > > > clever
> > > > > > way
> > > > > > > to do it by fetching the data, but not having the brokers act on it
> > > > > until
> > > > > > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> > > > > discuss
> > > > > > > this more in a separate KIP that just discusses just Raft.
> > > > > > >
> > > > > > > >
> > > > > > > > In section "Broker Metadata Management", you mention "the
> > > > controller
> > > > > > will
> > > > > > > > send a full metadata image rather than a series of deltas". This
> > > > KIP
> > > > > > > > doesn't go into the set of operations that need to be supported
> > > on
> > > > > top
> > > > > > of
> > > > > > > > Raft but it would be interested if this "full metadata image"
> > > could
> > > > > be
> > > > > > > > express also as deltas. For example, assuming we are replicating
> > > a
> > > > > map
> > > > > > > this
> > > > > > > > "full metadata image" could be a sequence of "put" operations
> > > > (znode
> > > > > > > create
> > > > > > > > to borrow ZK semantics).
> > > > > > >
> > > > > > > The full image can definitely be expressed as a sum of deltas.  At
> > > > some
> > > > > > > point, the number of deltas will get large enough that sending a
> > > full
> > > > > > image
> > > > > > > is better, though.  One question that we're still thinking about is
> > > > how
> > > > > > > much of this can be shared with generic Kafka log code, and how
> > > much
> > > > > > should
> > > > > > > be different.
> > > > > > >
> > > > > > > >
> > > > > > > > In section "Broker Metadata Management", you mention "This
> > > request
> > > > > will
> > > > > > > > double as a heartbeat, letting the controller know that the
> > > broker
> > > > is
> > > > > > > > alive". In section "Broker State Machine", you mention "The
> > > > > > MetadataFetch
> > > > > > > > API serves as this registration mechanism". Does this mean that
> > > the
> > > > > > > > MetadataFetch Request will optionally include broker
> > > configuration
> > > > > > > > information?
> > > > > > >
> > > > > > > I was originally thinking that the MetadataFetchRequest should
> > > > include
> > > > > > > broker configuration information.  Thinking about this more, maybe
> > > we
> > > > > > > should just have a special registration RPC that contains that
> > > > > > information,
> > > > > > > to avoid sending it over the wire all the time.
> > > > > > >
> > > > > > > > Does this also mean that MetadataFetch request will result in
> > > > > > > > a "write"/AppendEntries through the Raft replication protocol
> > > > before
> > > > > > you
> > > > > > > > can send the associated MetadataFetch Response?
> > > > > > >
> > > > > > > I think we should require the broker to be out of the Offline state
> > > > > > before
> > > > > > > allowing it to fetch metadata, yes.  So the separate registration
> > > RPC
> > > > > > > should have completed first.
> > > > > > >
> > > > > > > >
> > > > > > > > In section "Broker State", you mention that a broker can
> > > transition
> > > > > to
> > > > > > > > online after it is caught with the metadata. What do you mean by
> > > > > this?
> > > > > > > > Metadata is always changing. How does the broker know that it is
> > > > > caught
> > > > > > > up
> > > > > > > > since it doesn't participate in the consensus or the advancement
> > > of
> > > > > the
> > > > > > > > highwatermark?
> > > > > > >
> > > > > > > That's a good point.  Being "caught up" is somewhat of a fuzzy
> > > > concept
> > > > > > > here, since the brokers do not participate in the metadata
> > > consensus.
> > > > > I
> > > > > > > think ideally we would want to define it in terms of time ("the
> > > > broker
> > > > > > has
> > > > > > > all the updates from the last 2 minutes", for example.)  We should
> > > > > spell
> > > > > > > this out better in the KIP.
> > > > > > >
> > > > > > > >
> > > > > > > > In section "Start the controller quorum nodes", you mention "Once
> > > > it
> > > > > > has
> > > > > > > > taken over the /controller node, the active controller will
> > > proceed
> > > > > to
> > > > > > > load
> > > > > > > > the full state of ZooKeeper.  It will write out this information
> > > to
> > > > > the
> > > > > > > > quorum's metadata storage.  After this point, the metadata quorum
> > > > > will
> > > > > > be
> > > > > > > > the metadata store of record, rather than the data in ZooKeeper."
> > > > > > During
> > > > > > > > this migration do should we expect to have a small period
> > > > controller
> > > > > > > > unavailability while the controller replicas this state to all of
> > > > the
> > > > > > > raft
> > > > > > > > nodes in the controller quorum and we buffer new controller API
> > > > > > requests?
> > > > > > >
> > > > > > > Yes, the controller would be unavailable during this time.  I don't
> > > > > think
> > > > > > > this will be that different from the current period of
> > > unavailability
> > > > > > when
> > > > > > > a new controller starts up and needs to load the full state from
> > > ZK.
> > > > > The
> > > > > > > main difference is that in this period, we'd have to write to the
> > > > > > > controller quorum rather than just to memory.  But we believe this
> > > > > should
> > > > > > > be pretty fast.
> > > > > > >
> > > > > > > regards,
> > > > > > > Colin
> > > > > > >
> > > > > > > >
> > > > > > > > Thanks!
> > > > > > > > -Jose
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> Hi Colin,
> 
> Thanks for the KIP.
> 
> Currently ZooKeeper provides a convenient notification mechanism for
> knowing that broker and topic configuration has changed. While KIP-500 does
> suggest that incremental metadata update is expected to come to clients
> eventually, that would seem to imply that for some number of releases there
> would be no equivalent mechanism for knowing about config changes. Is there
> any thinking at this point about how a similar notification might be
> provided in the future?

We could eventually have some inotify-like mechanism where clients could register interest in various types of events and got notified when they happened.  Reading the metadata log is conceptually simple.  The main complexity would be in setting up an API that made sense and that didn't unduly constrain future implementations.  We'd have to think carefully about what the real use-cases for this were, though.

best,
Colin

> 
> Thanks,
> 
> Tom
> 
> On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <vi...@gmail.com>
> wrote:
> 
> > Hey Colin,
> >
> > I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> > see this in Kafka once. I collected my questions (and I accept the "TBD"
> > answer as they might be a bit deep for this high level :) ).
> > 1.) Are there any specific reasons for the Controller just periodically
> > persisting its state on disk periodically instead of asynchronously with
> > every update? Wouldn't less frequent saves increase the chance for missing
> > a state change if the controller crashes between two saves?
> > 2.) Why can't we allow brokers to fetch metadata from the follower
> > controllers? I assume that followers would have up-to-date information
> > therefore brokers could fetch from there in theory.
> >
> > Thanks,
> > Viktor
> >
> > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> > > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds
> > like
> > > a good idea.
> > >
> > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk> wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Yes, there will be several KIPs that will discuss the items you
> > describe
> > > in
> > > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > > description.
> > > >
> > > > Ismael
> > > >
> > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <reluctanthero104@gmail.com
> > >
> > > > wrote:
> > > >
> > > > > Thanks Colin for initiating this important effort!
> > > > >
> > > > > One question I have is whether we have a session discussing the
> > > > controller
> > > > > failover in the new architecture? I know we are using Raft protocol
> > to
> > > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > > going
> > > > > to take to reach the stable stage again, so that we could easily
> > > measure
> > > > > the availability of the metadata servers.
> > > > >
> > > > > Another suggestion I have is to write a step-by-step design doc like
> > > what
> > > > > we did in KIP-98
> > > > > <
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > >,
> > > > > including the new request protocols and how they are interacting in
> > the
> > > > new
> > > > > cluster. For a complicated change like this, an implementation design
> > > doc
> > > > > help a lot in the review process, otherwise most discussions we have
> > > will
> > > > > focus on high level and lose important details as we discover them in
> > > the
> > > > > post-agreement phase.
> > > > >
> > > > > Boyang
> > > > >
> > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org>
> > > wrote:
> > > > >
> > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > > questions.
> > > > > > >
> > > > > > > In the KIP's Motivation and Overview you mentioned the
> > LeaderAndIsr
> > > > and
> > > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > > pushes,
> > > > > > such
> > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking
> > that
> > > > we
> > > > > > will
> > > > > > > use MetadataFetch as a replacement to just UpdateMetadata only
> > and
> > > > add
> > > > > > > topic configuration in this state?
> > > > > > >
> > > > > >
> > > > > > Hi Jose,
> > > > > >
> > > > > > Thanks for taking a look.
> > > > > >
> > > > > > The goal is for MetadataFetchRequest to replace both
> > > > LeaderAndIsrRequest
> > > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > > along
> > > > > > with the other metadata.
> > > > > >
> > > > > > > In the section "Broker Metadata Management", you mention "Just
> > like
> > > > > with
> > > > > > a
> > > > > > > fetch request, the broker will track the offset of the last
> > updates
> > > > it
> > > > > > > fetched". To keep the log consistent Raft requires that the
> > > followers
> > > > > > keep
> > > > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> > > return
> > > > > log
> > > > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > > > which
> > > > > > > will replicate/fetch log entries after the highwatermark for
> > > > consensus?
> > > > > >
> > > > > > Good question.  Clearly, we shouldn't expose metadata updates to
> > the
> > > > > > brokers until they've been stored on a majority of the Raft nodes.
> > > The
> > > > > > most obvious way to do that, like you mentioned, is to have the
> > > brokers
> > > > > > only fetch up to the HWM, but not beyond.  There might be a more
> > > clever
> > > > > way
> > > > > > to do it by fetching the data, but not having the brokers act on it
> > > > until
> > > > > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> > > > discuss
> > > > > > this more in a separate KIP that just discusses just Raft.
> > > > > >
> > > > > > >
> > > > > > > In section "Broker Metadata Management", you mention "the
> > > controller
> > > > > will
> > > > > > > send a full metadata image rather than a series of deltas". This
> > > KIP
> > > > > > > doesn't go into the set of operations that need to be supported
> > on
> > > > top
> > > > > of
> > > > > > > Raft but it would be interested if this "full metadata image"
> > could
> > > > be
> > > > > > > express also as deltas. For example, assuming we are replicating
> > a
> > > > map
> > > > > > this
> > > > > > > "full metadata image" could be a sequence of "put" operations
> > > (znode
> > > > > > create
> > > > > > > to borrow ZK semantics).
> > > > > >
> > > > > > The full image can definitely be expressed as a sum of deltas.  At
> > > some
> > > > > > point, the number of deltas will get large enough that sending a
> > full
> > > > > image
> > > > > > is better, though.  One question that we're still thinking about is
> > > how
> > > > > > much of this can be shared with generic Kafka log code, and how
> > much
> > > > > should
> > > > > > be different.
> > > > > >
> > > > > > >
> > > > > > > In section "Broker Metadata Management", you mention "This
> > request
> > > > will
> > > > > > > double as a heartbeat, letting the controller know that the
> > broker
> > > is
> > > > > > > alive". In section "Broker State Machine", you mention "The
> > > > > MetadataFetch
> > > > > > > API serves as this registration mechanism". Does this mean that
> > the
> > > > > > > MetadataFetch Request will optionally include broker
> > configuration
> > > > > > > information?
> > > > > >
> > > > > > I was originally thinking that the MetadataFetchRequest should
> > > include
> > > > > > broker configuration information.  Thinking about this more, maybe
> > we
> > > > > > should just have a special registration RPC that contains that
> > > > > information,
> > > > > > to avoid sending it over the wire all the time.
> > > > > >
> > > > > > > Does this also mean that MetadataFetch request will result in
> > > > > > > a "write"/AppendEntries through the Raft replication protocol
> > > before
> > > > > you
> > > > > > > can send the associated MetadataFetch Response?
> > > > > >
> > > > > > I think we should require the broker to be out of the Offline state
> > > > > before
> > > > > > allowing it to fetch metadata, yes.  So the separate registration
> > RPC
> > > > > > should have completed first.
> > > > > >
> > > > > > >
> > > > > > > In section "Broker State", you mention that a broker can
> > transition
> > > > to
> > > > > > > online after it is caught with the metadata. What do you mean by
> > > > this?
> > > > > > > Metadata is always changing. How does the broker know that it is
> > > > caught
> > > > > > up
> > > > > > > since it doesn't participate in the consensus or the advancement
> > of
> > > > the
> > > > > > > highwatermark?
> > > > > >
> > > > > > That's a good point.  Being "caught up" is somewhat of a fuzzy
> > > concept
> > > > > > here, since the brokers do not participate in the metadata
> > consensus.
> > > > I
> > > > > > think ideally we would want to define it in terms of time ("the
> > > broker
> > > > > has
> > > > > > all the updates from the last 2 minutes", for example.)  We should
> > > > spell
> > > > > > this out better in the KIP.
> > > > > >
> > > > > > >
> > > > > > > In section "Start the controller quorum nodes", you mention "Once
> > > it
> > > > > has
> > > > > > > taken over the /controller node, the active controller will
> > proceed
> > > > to
> > > > > > load
> > > > > > > the full state of ZooKeeper.  It will write out this information
> > to
> > > > the
> > > > > > > quorum's metadata storage.  After this point, the metadata quorum
> > > > will
> > > > > be
> > > > > > > the metadata store of record, rather than the data in ZooKeeper."
> > > > > During
> > > > > > > this migration do should we expect to have a small period
> > > controller
> > > > > > > unavailability while the controller replicas this state to all of
> > > the
> > > > > > raft
> > > > > > > nodes in the controller quorum and we buffer new controller API
> > > > > requests?
> > > > > >
> > > > > > Yes, the controller would be unavailable during this time.  I don't
> > > > think
> > > > > > this will be that different from the current period of
> > unavailability
> > > > > when
> > > > > > a new controller starts up and needs to load the full state from
> > ZK.
> > > > The
> > > > > > main difference is that in this period, we'd have to write to the
> > > > > > controller quorum rather than just to memory.  But we believe this
> > > > should
> > > > > > be pretty fast.
> > > > > >
> > > > > > regards,
> > > > > > Colin
> > > > > >
> > > > > > >
> > > > > > > Thanks!
> > > > > > > -Jose
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Tom Bentley <tb...@redhat.com>.
Hi Colin,

Thanks for the KIP.

Currently ZooKeeper provides a convenient notification mechanism for
knowing that broker and topic configuration has changed. While KIP-500 does
suggest that incremental metadata update is expected to come to clients
eventually, that would seem to imply that for some number of releases there
would be no equivalent mechanism for knowing about config changes. Is there
any thinking at this point about how a similar notification might be
provided in the future?

Thanks,

Tom

On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass <vi...@gmail.com>
wrote:

> Hey Colin,
>
> I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> see this in Kafka once. I collected my questions (and I accept the "TBD"
> answer as they might be a bit deep for this high level :) ).
> 1.) Are there any specific reasons for the Controller just periodically
> persisting its state on disk periodically instead of asynchronously with
> every update? Wouldn't less frequent saves increase the chance for missing
> a state change if the controller crashes between two saves?
> 2.) Why can't we allow brokers to fetch metadata from the follower
> controllers? I assume that followers would have up-to-date information
> therefore brokers could fetch from there in theory.
>
> Thanks,
> Viktor
>
> On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <re...@gmail.com>
> wrote:
>
> > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds
> like
> > a good idea.
> >
> > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Hi Boyang,
> > >
> > > Yes, there will be several KIPs that will discuss the items you
> describe
> > in
> > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > description.
> > >
> > > Ismael
> > >
> > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <reluctanthero104@gmail.com
> >
> > > wrote:
> > >
> > > > Thanks Colin for initiating this important effort!
> > > >
> > > > One question I have is whether we have a session discussing the
> > > controller
> > > > failover in the new architecture? I know we are using Raft protocol
> to
> > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > going
> > > > to take to reach the stable stage again, so that we could easily
> > measure
> > > > the availability of the metadata servers.
> > > >
> > > > Another suggestion I have is to write a step-by-step design doc like
> > what
> > > > we did in KIP-98
> > > > <
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >,
> > > > including the new request protocols and how they are interacting in
> the
> > > new
> > > > cluster. For a complicated change like this, an implementation design
> > doc
> > > > help a lot in the review process, otherwise most discussions we have
> > will
> > > > focus on high level and lose important details as we discover them in
> > the
> > > > post-agreement phase.
> > > >
> > > > Boyang
> > > >
> > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > >
> > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > questions.
> > > > > >
> > > > > > In the KIP's Motivation and Overview you mentioned the
> LeaderAndIsr
> > > and
> > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > pushes,
> > > > > such
> > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking
> that
> > > we
> > > > > will
> > > > > > use MetadataFetch as a replacement to just UpdateMetadata only
> and
> > > add
> > > > > > topic configuration in this state?
> > > > > >
> > > > >
> > > > > Hi Jose,
> > > > >
> > > > > Thanks for taking a look.
> > > > >
> > > > > The goal is for MetadataFetchRequest to replace both
> > > LeaderAndIsrRequest
> > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > along
> > > > > with the other metadata.
> > > > >
> > > > > > In the section "Broker Metadata Management", you mention "Just
> like
> > > > with
> > > > > a
> > > > > > fetch request, the broker will track the offset of the last
> updates
> > > it
> > > > > > fetched". To keep the log consistent Raft requires that the
> > followers
> > > > > keep
> > > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> > return
> > > > log
> > > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > > which
> > > > > > will replicate/fetch log entries after the highwatermark for
> > > consensus?
> > > > >
> > > > > Good question.  Clearly, we shouldn't expose metadata updates to
> the
> > > > > brokers until they've been stored on a majority of the Raft nodes.
> > The
> > > > > most obvious way to do that, like you mentioned, is to have the
> > brokers
> > > > > only fetch up to the HWM, but not beyond.  There might be a more
> > clever
> > > > way
> > > > > to do it by fetching the data, but not having the brokers act on it
> > > until
> > > > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> > > discuss
> > > > > this more in a separate KIP that just discusses just Raft.
> > > > >
> > > > > >
> > > > > > In section "Broker Metadata Management", you mention "the
> > controller
> > > > will
> > > > > > send a full metadata image rather than a series of deltas". This
> > KIP
> > > > > > doesn't go into the set of operations that need to be supported
> on
> > > top
> > > > of
> > > > > > Raft but it would be interested if this "full metadata image"
> could
> > > be
> > > > > > express also as deltas. For example, assuming we are replicating
> a
> > > map
> > > > > this
> > > > > > "full metadata image" could be a sequence of "put" operations
> > (znode
> > > > > create
> > > > > > to borrow ZK semantics).
> > > > >
> > > > > The full image can definitely be expressed as a sum of deltas.  At
> > some
> > > > > point, the number of deltas will get large enough that sending a
> full
> > > > image
> > > > > is better, though.  One question that we're still thinking about is
> > how
> > > > > much of this can be shared with generic Kafka log code, and how
> much
> > > > should
> > > > > be different.
> > > > >
> > > > > >
> > > > > > In section "Broker Metadata Management", you mention "This
> request
> > > will
> > > > > > double as a heartbeat, letting the controller know that the
> broker
> > is
> > > > > > alive". In section "Broker State Machine", you mention "The
> > > > MetadataFetch
> > > > > > API serves as this registration mechanism". Does this mean that
> the
> > > > > > MetadataFetch Request will optionally include broker
> configuration
> > > > > > information?
> > > > >
> > > > > I was originally thinking that the MetadataFetchRequest should
> > include
> > > > > broker configuration information.  Thinking about this more, maybe
> we
> > > > > should just have a special registration RPC that contains that
> > > > information,
> > > > > to avoid sending it over the wire all the time.
> > > > >
> > > > > > Does this also mean that MetadataFetch request will result in
> > > > > > a "write"/AppendEntries through the Raft replication protocol
> > before
> > > > you
> > > > > > can send the associated MetadataFetch Response?
> > > > >
> > > > > I think we should require the broker to be out of the Offline state
> > > > before
> > > > > allowing it to fetch metadata, yes.  So the separate registration
> RPC
> > > > > should have completed first.
> > > > >
> > > > > >
> > > > > > In section "Broker State", you mention that a broker can
> transition
> > > to
> > > > > > online after it is caught with the metadata. What do you mean by
> > > this?
> > > > > > Metadata is always changing. How does the broker know that it is
> > > caught
> > > > > up
> > > > > > since it doesn't participate in the consensus or the advancement
> of
> > > the
> > > > > > highwatermark?
> > > > >
> > > > > That's a good point.  Being "caught up" is somewhat of a fuzzy
> > concept
> > > > > here, since the brokers do not participate in the metadata
> consensus.
> > > I
> > > > > think ideally we would want to define it in terms of time ("the
> > broker
> > > > has
> > > > > all the updates from the last 2 minutes", for example.)  We should
> > > spell
> > > > > this out better in the KIP.
> > > > >
> > > > > >
> > > > > > In section "Start the controller quorum nodes", you mention "Once
> > it
> > > > has
> > > > > > taken over the /controller node, the active controller will
> proceed
> > > to
> > > > > load
> > > > > > the full state of ZooKeeper.  It will write out this information
> to
> > > the
> > > > > > quorum's metadata storage.  After this point, the metadata quorum
> > > will
> > > > be
> > > > > > the metadata store of record, rather than the data in ZooKeeper."
> > > > During
> > > > > > this migration do should we expect to have a small period
> > controller
> > > > > > unavailability while the controller replicas this state to all of
> > the
> > > > > raft
> > > > > > nodes in the controller quorum and we buffer new controller API
> > > > requests?
> > > > >
> > > > > Yes, the controller would be unavailable during this time.  I don't
> > > think
> > > > > this will be that different from the current period of
> unavailability
> > > > when
> > > > > a new controller starts up and needs to load the full state from
> ZK.
> > > The
> > > > > main difference is that in this period, we'd have to write to the
> > > > > controller quorum rather than just to memory.  But we believe this
> > > should
> > > > > be pretty fast.
> > > > >
> > > > > regards,
> > > > > Colin
> > > > >
> > > > > >
> > > > > > Thanks!
> > > > > > -Jose
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Mon, Aug 5, 2019, at 07:49, Viktor Somogyi-Vass wrote:
> Hey Colin,
> 
> I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> see this in Kafka once. I collected my questions (and I accept the "TBD"
> answer as they might be a bit deep for this high level :) ).
> 1.) Are there any specific reasons for the Controller just periodically
> persisting its state on disk periodically instead of asynchronously with
> every update? Wouldn't less frequent saves increase the chance for missing
> a state change if the controller crashes between two saves?

Hi Viktor,

To be clear, the controller will persist its state on disk with every update.  It's just that the state will be persisted in the form of a write-ahead log entry.  The periodic snapshots are to shorten loading times, not a mechanism for durability.

> 2.) Why can't we allow brokers to fetch metadata from the follower
> controllers? I assume that followers would have up-to-date information
> therefore brokers could fetch from there in theory.

In Raft, the leader always has the most up-to-date information.  A follower may not have up-to-date information.  For example, it may not have been part of the majority that wrote the latest update.

We also want the controller to know when brokers stop fetching metadata, so that they can be transitioned to a fenced state.  This requires fetching from the controller.

best,
Colin

> 
> Thanks,
> Viktor
> 
> On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <re...@gmail.com>
> wrote:
> 
> > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds like
> > a good idea.
> >
> > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Hi Boyang,
> > >
> > > Yes, there will be several KIPs that will discuss the items you describe
> > in
> > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > description.
> > >
> > > Ismael
> > >
> > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <re...@gmail.com>
> > > wrote:
> > >
> > > > Thanks Colin for initiating this important effort!
> > > >
> > > > One question I have is whether we have a session discussing the
> > > controller
> > > > failover in the new architecture? I know we are using Raft protocol to
> > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > going
> > > > to take to reach the stable stage again, so that we could easily
> > measure
> > > > the availability of the metadata servers.
> > > >
> > > > Another suggestion I have is to write a step-by-step design doc like
> > what
> > > > we did in KIP-98
> > > > <
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >,
> > > > including the new request protocols and how they are interacting in the
> > > new
> > > > cluster. For a complicated change like this, an implementation design
> > doc
> > > > help a lot in the review process, otherwise most discussions we have
> > will
> > > > focus on high level and lose important details as we discover them in
> > the
> > > > post-agreement phase.
> > > >
> > > > Boyang
> > > >
> > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org>
> > wrote:
> > > >
> > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > questions.
> > > > > >
> > > > > > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr
> > > and
> > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > pushes,
> > > > > such
> > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking that
> > > we
> > > > > will
> > > > > > use MetadataFetch as a replacement to just UpdateMetadata only and
> > > add
> > > > > > topic configuration in this state?
> > > > > >
> > > > >
> > > > > Hi Jose,
> > > > >
> > > > > Thanks for taking a look.
> > > > >
> > > > > The goal is for MetadataFetchRequest to replace both
> > > LeaderAndIsrRequest
> > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > along
> > > > > with the other metadata.
> > > > >
> > > > > > In the section "Broker Metadata Management", you mention "Just like
> > > > with
> > > > > a
> > > > > > fetch request, the broker will track the offset of the last updates
> > > it
> > > > > > fetched". To keep the log consistent Raft requires that the
> > followers
> > > > > keep
> > > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> > return
> > > > log
> > > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > > which
> > > > > > will replicate/fetch log entries after the highwatermark for
> > > consensus?
> > > > >
> > > > > Good question.  Clearly, we shouldn't expose metadata updates to the
> > > > > brokers until they've been stored on a majority of the Raft nodes.
> > The
> > > > > most obvious way to do that, like you mentioned, is to have the
> > brokers
> > > > > only fetch up to the HWM, but not beyond.  There might be a more
> > clever
> > > > way
> > > > > to do it by fetching the data, but not having the brokers act on it
> > > until
> > > > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> > > discuss
> > > > > this more in a separate KIP that just discusses just Raft.
> > > > >
> > > > > >
> > > > > > In section "Broker Metadata Management", you mention "the
> > controller
> > > > will
> > > > > > send a full metadata image rather than a series of deltas". This
> > KIP
> > > > > > doesn't go into the set of operations that need to be supported on
> > > top
> > > > of
> > > > > > Raft but it would be interested if this "full metadata image" could
> > > be
> > > > > > express also as deltas. For example, assuming we are replicating a
> > > map
> > > > > this
> > > > > > "full metadata image" could be a sequence of "put" operations
> > (znode
> > > > > create
> > > > > > to borrow ZK semantics).
> > > > >
> > > > > The full image can definitely be expressed as a sum of deltas.  At
> > some
> > > > > point, the number of deltas will get large enough that sending a full
> > > > image
> > > > > is better, though.  One question that we're still thinking about is
> > how
> > > > > much of this can be shared with generic Kafka log code, and how much
> > > > should
> > > > > be different.
> > > > >
> > > > > >
> > > > > > In section "Broker Metadata Management", you mention "This request
> > > will
> > > > > > double as a heartbeat, letting the controller know that the broker
> > is
> > > > > > alive". In section "Broker State Machine", you mention "The
> > > > MetadataFetch
> > > > > > API serves as this registration mechanism". Does this mean that the
> > > > > > MetadataFetch Request will optionally include broker configuration
> > > > > > information?
> > > > >
> > > > > I was originally thinking that the MetadataFetchRequest should
> > include
> > > > > broker configuration information.  Thinking about this more, maybe we
> > > > > should just have a special registration RPC that contains that
> > > > information,
> > > > > to avoid sending it over the wire all the time.
> > > > >
> > > > > > Does this also mean that MetadataFetch request will result in
> > > > > > a "write"/AppendEntries through the Raft replication protocol
> > before
> > > > you
> > > > > > can send the associated MetadataFetch Response?
> > > > >
> > > > > I think we should require the broker to be out of the Offline state
> > > > before
> > > > > allowing it to fetch metadata, yes.  So the separate registration RPC
> > > > > should have completed first.
> > > > >
> > > > > >
> > > > > > In section "Broker State", you mention that a broker can transition
> > > to
> > > > > > online after it is caught with the metadata. What do you mean by
> > > this?
> > > > > > Metadata is always changing. How does the broker know that it is
> > > caught
> > > > > up
> > > > > > since it doesn't participate in the consensus or the advancement of
> > > the
> > > > > > highwatermark?
> > > > >
> > > > > That's a good point.  Being "caught up" is somewhat of a fuzzy
> > concept
> > > > > here, since the brokers do not participate in the metadata consensus.
> > > I
> > > > > think ideally we would want to define it in terms of time ("the
> > broker
> > > > has
> > > > > all the updates from the last 2 minutes", for example.)  We should
> > > spell
> > > > > this out better in the KIP.
> > > > >
> > > > > >
> > > > > > In section "Start the controller quorum nodes", you mention "Once
> > it
> > > > has
> > > > > > taken over the /controller node, the active controller will proceed
> > > to
> > > > > load
> > > > > > the full state of ZooKeeper.  It will write out this information to
> > > the
> > > > > > quorum's metadata storage.  After this point, the metadata quorum
> > > will
> > > > be
> > > > > > the metadata store of record, rather than the data in ZooKeeper."
> > > > During
> > > > > > this migration do should we expect to have a small period
> > controller
> > > > > > unavailability while the controller replicas this state to all of
> > the
> > > > > raft
> > > > > > nodes in the controller quorum and we buffer new controller API
> > > > requests?
> > > > >
> > > > > Yes, the controller would be unavailable during this time.  I don't
> > > think
> > > > > this will be that different from the current period of unavailability
> > > > when
> > > > > a new controller starts up and needs to load the full state from ZK.
> > > The
> > > > > main difference is that in this period, we'd have to write to the
> > > > > controller quorum rather than just to memory.  But we believe this
> > > should
> > > > > be pretty fast.
> > > > >
> > > > > regards,
> > > > > Colin
> > > > >
> > > > > >
> > > > > > Thanks!
> > > > > > -Jose
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Viktor Somogyi-Vass <vi...@gmail.com>.
Hey Colin,

I think this is a long-awaited KIP, thanks for driving it. I'm excited to
see this in Kafka once. I collected my questions (and I accept the "TBD"
answer as they might be a bit deep for this high level :) ).
1.) Are there any specific reasons for the Controller just periodically
persisting its state on disk periodically instead of asynchronously with
every update? Wouldn't less frequent saves increase the chance for missing
a state change if the controller crashes between two saves?
2.) Why can't we allow brokers to fetch metadata from the follower
controllers? I assume that followers would have up-to-date information
therefore brokers could fetch from there in theory.

Thanks,
Viktor

On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen <re...@gmail.com>
wrote:

> Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds like
> a good idea.
>
> On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk> wrote:
>
> > Hi Boyang,
> >
> > Yes, there will be several KIPs that will discuss the items you describe
> in
> > detail. Colin, it may be helpful to make this clear in the KIP 500
> > description.
> >
> > Ismael
> >
> > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <re...@gmail.com>
> > wrote:
> >
> > > Thanks Colin for initiating this important effort!
> > >
> > > One question I have is whether we have a session discussing the
> > controller
> > > failover in the new architecture? I know we are using Raft protocol to
> > > failover, yet it's still valuable to discuss the steps new cluster is
> > going
> > > to take to reach the stable stage again, so that we could easily
> measure
> > > the availability of the metadata servers.
> > >
> > > Another suggestion I have is to write a step-by-step design doc like
> what
> > > we did in KIP-98
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >,
> > > including the new request protocols and how they are interacting in the
> > new
> > > cluster. For a complicated change like this, an implementation design
> doc
> > > help a lot in the review process, otherwise most discussions we have
> will
> > > focus on high level and lose important details as we discover them in
> the
> > > post-agreement phase.
> > >
> > > Boyang
> > >
> > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org>
> wrote:
> > >
> > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > Thanks Colin for the detail KIP. I have a few comments and
> questions.
> > > > >
> > > > > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr
> > and
> > > > > UpdateMetadata RPC. For example, "updates which the controller
> > pushes,
> > > > such
> > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking that
> > we
> > > > will
> > > > > use MetadataFetch as a replacement to just UpdateMetadata only and
> > add
> > > > > topic configuration in this state?
> > > > >
> > > >
> > > > Hi Jose,
> > > >
> > > > Thanks for taking a look.
> > > >
> > > > The goal is for MetadataFetchRequest to replace both
> > LeaderAndIsrRequest
> > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> along
> > > > with the other metadata.
> > > >
> > > > > In the section "Broker Metadata Management", you mention "Just like
> > > with
> > > > a
> > > > > fetch request, the broker will track the offset of the last updates
> > it
> > > > > fetched". To keep the log consistent Raft requires that the
> followers
> > > > keep
> > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> return
> > > log
> > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > which
> > > > > will replicate/fetch log entries after the highwatermark for
> > consensus?
> > > >
> > > > Good question.  Clearly, we shouldn't expose metadata updates to the
> > > > brokers until they've been stored on a majority of the Raft nodes.
> The
> > > > most obvious way to do that, like you mentioned, is to have the
> brokers
> > > > only fetch up to the HWM, but not beyond.  There might be a more
> clever
> > > way
> > > > to do it by fetching the data, but not having the brokers act on it
> > until
> > > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> > discuss
> > > > this more in a separate KIP that just discusses just Raft.
> > > >
> > > > >
> > > > > In section "Broker Metadata Management", you mention "the
> controller
> > > will
> > > > > send a full metadata image rather than a series of deltas". This
> KIP
> > > > > doesn't go into the set of operations that need to be supported on
> > top
> > > of
> > > > > Raft but it would be interested if this "full metadata image" could
> > be
> > > > > express also as deltas. For example, assuming we are replicating a
> > map
> > > > this
> > > > > "full metadata image" could be a sequence of "put" operations
> (znode
> > > > create
> > > > > to borrow ZK semantics).
> > > >
> > > > The full image can definitely be expressed as a sum of deltas.  At
> some
> > > > point, the number of deltas will get large enough that sending a full
> > > image
> > > > is better, though.  One question that we're still thinking about is
> how
> > > > much of this can be shared with generic Kafka log code, and how much
> > > should
> > > > be different.
> > > >
> > > > >
> > > > > In section "Broker Metadata Management", you mention "This request
> > will
> > > > > double as a heartbeat, letting the controller know that the broker
> is
> > > > > alive". In section "Broker State Machine", you mention "The
> > > MetadataFetch
> > > > > API serves as this registration mechanism". Does this mean that the
> > > > > MetadataFetch Request will optionally include broker configuration
> > > > > information?
> > > >
> > > > I was originally thinking that the MetadataFetchRequest should
> include
> > > > broker configuration information.  Thinking about this more, maybe we
> > > > should just have a special registration RPC that contains that
> > > information,
> > > > to avoid sending it over the wire all the time.
> > > >
> > > > > Does this also mean that MetadataFetch request will result in
> > > > > a "write"/AppendEntries through the Raft replication protocol
> before
> > > you
> > > > > can send the associated MetadataFetch Response?
> > > >
> > > > I think we should require the broker to be out of the Offline state
> > > before
> > > > allowing it to fetch metadata, yes.  So the separate registration RPC
> > > > should have completed first.
> > > >
> > > > >
> > > > > In section "Broker State", you mention that a broker can transition
> > to
> > > > > online after it is caught with the metadata. What do you mean by
> > this?
> > > > > Metadata is always changing. How does the broker know that it is
> > caught
> > > > up
> > > > > since it doesn't participate in the consensus or the advancement of
> > the
> > > > > highwatermark?
> > > >
> > > > That's a good point.  Being "caught up" is somewhat of a fuzzy
> concept
> > > > here, since the brokers do not participate in the metadata consensus.
> > I
> > > > think ideally we would want to define it in terms of time ("the
> broker
> > > has
> > > > all the updates from the last 2 minutes", for example.)  We should
> > spell
> > > > this out better in the KIP.
> > > >
> > > > >
> > > > > In section "Start the controller quorum nodes", you mention "Once
> it
> > > has
> > > > > taken over the /controller node, the active controller will proceed
> > to
> > > > load
> > > > > the full state of ZooKeeper.  It will write out this information to
> > the
> > > > > quorum's metadata storage.  After this point, the metadata quorum
> > will
> > > be
> > > > > the metadata store of record, rather than the data in ZooKeeper."
> > > During
> > > > > this migration do should we expect to have a small period
> controller
> > > > > unavailability while the controller replicas this state to all of
> the
> > > > raft
> > > > > nodes in the controller quorum and we buffer new controller API
> > > requests?
> > > >
> > > > Yes, the controller would be unavailable during this time.  I don't
> > think
> > > > this will be that different from the current period of unavailability
> > > when
> > > > a new controller starts up and needs to load the full state from ZK.
> > The
> > > > main difference is that in this period, we'd have to write to the
> > > > controller quorum rather than just to memory.  But we believe this
> > should
> > > > be pretty fast.
> > > >
> > > > regards,
> > > > Colin
> > > >
> > > > >
> > > > > Thanks!
> > > > > -Jose
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Boyang Chen <re...@gmail.com>.
Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds like
a good idea.

On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma <is...@juma.me.uk> wrote:

> Hi Boyang,
>
> Yes, there will be several KIPs that will discuss the items you describe in
> detail. Colin, it may be helpful to make this clear in the KIP 500
> description.
>
> Ismael
>
> On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <re...@gmail.com>
> wrote:
>
> > Thanks Colin for initiating this important effort!
> >
> > One question I have is whether we have a session discussing the
> controller
> > failover in the new architecture? I know we are using Raft protocol to
> > failover, yet it's still valuable to discuss the steps new cluster is
> going
> > to take to reach the stable stage again, so that we could easily measure
> > the availability of the metadata servers.
> >
> > Another suggestion I have is to write a step-by-step design doc like what
> > we did in KIP-98
> > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >,
> > including the new request protocols and how they are interacting in the
> new
> > cluster. For a complicated change like this, an implementation design doc
> > help a lot in the review process, otherwise most discussions we have will
> > focus on high level and lose important details as we discover them in the
> > post-agreement phase.
> >
> > Boyang
> >
> > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > Thanks Colin for the detail KIP. I have a few comments and questions.
> > > >
> > > > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr
> and
> > > > UpdateMetadata RPC. For example, "updates which the controller
> pushes,
> > > such
> > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking that
> we
> > > will
> > > > use MetadataFetch as a replacement to just UpdateMetadata only and
> add
> > > > topic configuration in this state?
> > > >
> > >
> > > Hi Jose,
> > >
> > > Thanks for taking a look.
> > >
> > > The goal is for MetadataFetchRequest to replace both
> LeaderAndIsrRequest
> > > and UpdateMetadataRequest.  Topic configurations would be fetched along
> > > with the other metadata.
> > >
> > > > In the section "Broker Metadata Management", you mention "Just like
> > with
> > > a
> > > > fetch request, the broker will track the offset of the last updates
> it
> > > > fetched". To keep the log consistent Raft requires that the followers
> > > keep
> > > > all of the log entries (term/epoch and offset) that are after the
> > > > highwatermark. Any log entry before the highwatermark can be
> > > > compacted/snapshot. Do we expect the MetadataFetch API to only return
> > log
> > > > entries up to the highwatermark?  Unlike the Raft replication API
> which
> > > > will replicate/fetch log entries after the highwatermark for
> consensus?
> > >
> > > Good question.  Clearly, we shouldn't expose metadata updates to the
> > > brokers until they've been stored on a majority of the Raft nodes.  The
> > > most obvious way to do that, like you mentioned, is to have the brokers
> > > only fetch up to the HWM, but not beyond.  There might be a more clever
> > way
> > > to do it by fetching the data, but not having the brokers act on it
> until
> > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> discuss
> > > this more in a separate KIP that just discusses just Raft.
> > >
> > > >
> > > > In section "Broker Metadata Management", you mention "the controller
> > will
> > > > send a full metadata image rather than a series of deltas". This KIP
> > > > doesn't go into the set of operations that need to be supported on
> top
> > of
> > > > Raft but it would be interested if this "full metadata image" could
> be
> > > > express also as deltas. For example, assuming we are replicating a
> map
> > > this
> > > > "full metadata image" could be a sequence of "put" operations (znode
> > > create
> > > > to borrow ZK semantics).
> > >
> > > The full image can definitely be expressed as a sum of deltas.  At some
> > > point, the number of deltas will get large enough that sending a full
> > image
> > > is better, though.  One question that we're still thinking about is how
> > > much of this can be shared with generic Kafka log code, and how much
> > should
> > > be different.
> > >
> > > >
> > > > In section "Broker Metadata Management", you mention "This request
> will
> > > > double as a heartbeat, letting the controller know that the broker is
> > > > alive". In section "Broker State Machine", you mention "The
> > MetadataFetch
> > > > API serves as this registration mechanism". Does this mean that the
> > > > MetadataFetch Request will optionally include broker configuration
> > > > information?
> > >
> > > I was originally thinking that the MetadataFetchRequest should include
> > > broker configuration information.  Thinking about this more, maybe we
> > > should just have a special registration RPC that contains that
> > information,
> > > to avoid sending it over the wire all the time.
> > >
> > > > Does this also mean that MetadataFetch request will result in
> > > > a "write"/AppendEntries through the Raft replication protocol before
> > you
> > > > can send the associated MetadataFetch Response?
> > >
> > > I think we should require the broker to be out of the Offline state
> > before
> > > allowing it to fetch metadata, yes.  So the separate registration RPC
> > > should have completed first.
> > >
> > > >
> > > > In section "Broker State", you mention that a broker can transition
> to
> > > > online after it is caught with the metadata. What do you mean by
> this?
> > > > Metadata is always changing. How does the broker know that it is
> caught
> > > up
> > > > since it doesn't participate in the consensus or the advancement of
> the
> > > > highwatermark?
> > >
> > > That's a good point.  Being "caught up" is somewhat of a fuzzy concept
> > > here, since the brokers do not participate in the metadata consensus.
> I
> > > think ideally we would want to define it in terms of time ("the broker
> > has
> > > all the updates from the last 2 minutes", for example.)  We should
> spell
> > > this out better in the KIP.
> > >
> > > >
> > > > In section "Start the controller quorum nodes", you mention "Once it
> > has
> > > > taken over the /controller node, the active controller will proceed
> to
> > > load
> > > > the full state of ZooKeeper.  It will write out this information to
> the
> > > > quorum's metadata storage.  After this point, the metadata quorum
> will
> > be
> > > > the metadata store of record, rather than the data in ZooKeeper."
> > During
> > > > this migration do should we expect to have a small period controller
> > > > unavailability while the controller replicas this state to all of the
> > > raft
> > > > nodes in the controller quorum and we buffer new controller API
> > requests?
> > >
> > > Yes, the controller would be unavailable during this time.  I don't
> think
> > > this will be that different from the current period of unavailability
> > when
> > > a new controller starts up and needs to load the full state from ZK.
> The
> > > main difference is that in this period, we'd have to write to the
> > > controller quorum rather than just to memory.  But we believe this
> should
> > > be pretty fast.
> > >
> > > regards,
> > > Colin
> > >
> > > >
> > > > Thanks!
> > > > -Jose
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Boyang,

Yes, there will be several KIPs that will discuss the items you describe in
detail. Colin, it may be helpful to make this clear in the KIP 500
description.

Ismael

On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen <re...@gmail.com>
wrote:

> Thanks Colin for initiating this important effort!
>
> One question I have is whether we have a session discussing the controller
> failover in the new architecture? I know we are using Raft protocol to
> failover, yet it's still valuable to discuss the steps new cluster is going
> to take to reach the stable stage again, so that we could easily measure
> the availability of the metadata servers.
>
> Another suggestion I have is to write a step-by-step design doc like what
> we did in KIP-98
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >,
> including the new request protocols and how they are interacting in the new
> cluster. For a complicated change like this, an implementation design doc
> help a lot in the review process, otherwise most discussions we have will
> focus on high level and lose important details as we discover them in the
> post-agreement phase.
>
> Boyang
>
> On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org> wrote:
>
> > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > Thanks Colin for the detail KIP. I have a few comments and questions.
> > >
> > > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr and
> > > UpdateMetadata RPC. For example, "updates which the controller pushes,
> > such
> > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking that we
> > will
> > > use MetadataFetch as a replacement to just UpdateMetadata only and add
> > > topic configuration in this state?
> > >
> >
> > Hi Jose,
> >
> > Thanks for taking a look.
> >
> > The goal is for MetadataFetchRequest to replace both LeaderAndIsrRequest
> > and UpdateMetadataRequest.  Topic configurations would be fetched along
> > with the other metadata.
> >
> > > In the section "Broker Metadata Management", you mention "Just like
> with
> > a
> > > fetch request, the broker will track the offset of the last updates it
> > > fetched". To keep the log consistent Raft requires that the followers
> > keep
> > > all of the log entries (term/epoch and offset) that are after the
> > > highwatermark. Any log entry before the highwatermark can be
> > > compacted/snapshot. Do we expect the MetadataFetch API to only return
> log
> > > entries up to the highwatermark?  Unlike the Raft replication API which
> > > will replicate/fetch log entries after the highwatermark for consensus?
> >
> > Good question.  Clearly, we shouldn't expose metadata updates to the
> > brokers until they've been stored on a majority of the Raft nodes.  The
> > most obvious way to do that, like you mentioned, is to have the brokers
> > only fetch up to the HWM, but not beyond.  There might be a more clever
> way
> > to do it by fetching the data, but not having the brokers act on it until
> > the HWM advances.  I'm not sure if that's worth it or not.  We'll discuss
> > this more in a separate KIP that just discusses just Raft.
> >
> > >
> > > In section "Broker Metadata Management", you mention "the controller
> will
> > > send a full metadata image rather than a series of deltas". This KIP
> > > doesn't go into the set of operations that need to be supported on top
> of
> > > Raft but it would be interested if this "full metadata image" could be
> > > express also as deltas. For example, assuming we are replicating a map
> > this
> > > "full metadata image" could be a sequence of "put" operations (znode
> > create
> > > to borrow ZK semantics).
> >
> > The full image can definitely be expressed as a sum of deltas.  At some
> > point, the number of deltas will get large enough that sending a full
> image
> > is better, though.  One question that we're still thinking about is how
> > much of this can be shared with generic Kafka log code, and how much
> should
> > be different.
> >
> > >
> > > In section "Broker Metadata Management", you mention "This request will
> > > double as a heartbeat, letting the controller know that the broker is
> > > alive". In section "Broker State Machine", you mention "The
> MetadataFetch
> > > API serves as this registration mechanism". Does this mean that the
> > > MetadataFetch Request will optionally include broker configuration
> > > information?
> >
> > I was originally thinking that the MetadataFetchRequest should include
> > broker configuration information.  Thinking about this more, maybe we
> > should just have a special registration RPC that contains that
> information,
> > to avoid sending it over the wire all the time.
> >
> > > Does this also mean that MetadataFetch request will result in
> > > a "write"/AppendEntries through the Raft replication protocol before
> you
> > > can send the associated MetadataFetch Response?
> >
> > I think we should require the broker to be out of the Offline state
> before
> > allowing it to fetch metadata, yes.  So the separate registration RPC
> > should have completed first.
> >
> > >
> > > In section "Broker State", you mention that a broker can transition to
> > > online after it is caught with the metadata. What do you mean by this?
> > > Metadata is always changing. How does the broker know that it is caught
> > up
> > > since it doesn't participate in the consensus or the advancement of the
> > > highwatermark?
> >
> > That's a good point.  Being "caught up" is somewhat of a fuzzy concept
> > here, since the brokers do not participate in the metadata consensus.  I
> > think ideally we would want to define it in terms of time ("the broker
> has
> > all the updates from the last 2 minutes", for example.)  We should spell
> > this out better in the KIP.
> >
> > >
> > > In section "Start the controller quorum nodes", you mention "Once it
> has
> > > taken over the /controller node, the active controller will proceed to
> > load
> > > the full state of ZooKeeper.  It will write out this information to the
> > > quorum's metadata storage.  After this point, the metadata quorum will
> be
> > > the metadata store of record, rather than the data in ZooKeeper."
> During
> > > this migration do should we expect to have a small period controller
> > > unavailability while the controller replicas this state to all of the
> > raft
> > > nodes in the controller quorum and we buffer new controller API
> requests?
> >
> > Yes, the controller would be unavailable during this time.  I don't think
> > this will be that different from the current period of unavailability
> when
> > a new controller starts up and needs to load the full state from ZK.  The
> > main difference is that in this period, we'd have to write to the
> > controller quorum rather than just to memory.  But we believe this should
> > be pretty fast.
> >
> > regards,
> > Colin
> >
> > >
> > > Thanks!
> > > -Jose
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Boyang Chen <re...@gmail.com>.
Thanks Colin for initiating this important effort!

One question I have is whether we have a session discussing the controller
failover in the new architecture? I know we are using Raft protocol to
failover, yet it's still valuable to discuss the steps new cluster is going
to take to reach the stable stage again, so that we could easily measure
the availability of the metadata servers.

Another suggestion I have is to write a step-by-step design doc like what
we did in KIP-98
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging>,
including the new request protocols and how they are interacting in the new
cluster. For a complicated change like this, an implementation design doc
help a lot in the review process, otherwise most discussions we have will
focus on high level and lose important details as we discover them in the
post-agreement phase.

Boyang

On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe <cm...@apache.org> wrote:

> On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > Thanks Colin for the detail KIP. I have a few comments and questions.
> >
> > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr and
> > UpdateMetadata RPC. For example, "updates which the controller pushes,
> such
> > as LeaderAndIsr and UpdateMetadata messages". Is your thinking that we
> will
> > use MetadataFetch as a replacement to just UpdateMetadata only and add
> > topic configuration in this state?
> >
>
> Hi Jose,
>
> Thanks for taking a look.
>
> The goal is for MetadataFetchRequest to replace both LeaderAndIsrRequest
> and UpdateMetadataRequest.  Topic configurations would be fetched along
> with the other metadata.
>
> > In the section "Broker Metadata Management", you mention "Just like with
> a
> > fetch request, the broker will track the offset of the last updates it
> > fetched". To keep the log consistent Raft requires that the followers
> keep
> > all of the log entries (term/epoch and offset) that are after the
> > highwatermark. Any log entry before the highwatermark can be
> > compacted/snapshot. Do we expect the MetadataFetch API to only return log
> > entries up to the highwatermark?  Unlike the Raft replication API which
> > will replicate/fetch log entries after the highwatermark for consensus?
>
> Good question.  Clearly, we shouldn't expose metadata updates to the
> brokers until they've been stored on a majority of the Raft nodes.  The
> most obvious way to do that, like you mentioned, is to have the brokers
> only fetch up to the HWM, but not beyond.  There might be a more clever way
> to do it by fetching the data, but not having the brokers act on it until
> the HWM advances.  I'm not sure if that's worth it or not.  We'll discuss
> this more in a separate KIP that just discusses just Raft.
>
> >
> > In section "Broker Metadata Management", you mention "the controller will
> > send a full metadata image rather than a series of deltas". This KIP
> > doesn't go into the set of operations that need to be supported on top of
> > Raft but it would be interested if this "full metadata image" could be
> > express also as deltas. For example, assuming we are replicating a map
> this
> > "full metadata image" could be a sequence of "put" operations (znode
> create
> > to borrow ZK semantics).
>
> The full image can definitely be expressed as a sum of deltas.  At some
> point, the number of deltas will get large enough that sending a full image
> is better, though.  One question that we're still thinking about is how
> much of this can be shared with generic Kafka log code, and how much should
> be different.
>
> >
> > In section "Broker Metadata Management", you mention "This request will
> > double as a heartbeat, letting the controller know that the broker is
> > alive". In section "Broker State Machine", you mention "The MetadataFetch
> > API serves as this registration mechanism". Does this mean that the
> > MetadataFetch Request will optionally include broker configuration
> > information?
>
> I was originally thinking that the MetadataFetchRequest should include
> broker configuration information.  Thinking about this more, maybe we
> should just have a special registration RPC that contains that information,
> to avoid sending it over the wire all the time.
>
> > Does this also mean that MetadataFetch request will result in
> > a "write"/AppendEntries through the Raft replication protocol before you
> > can send the associated MetadataFetch Response?
>
> I think we should require the broker to be out of the Offline state before
> allowing it to fetch metadata, yes.  So the separate registration RPC
> should have completed first.
>
> >
> > In section "Broker State", you mention that a broker can transition to
> > online after it is caught with the metadata. What do you mean by this?
> > Metadata is always changing. How does the broker know that it is caught
> up
> > since it doesn't participate in the consensus or the advancement of the
> > highwatermark?
>
> That's a good point.  Being "caught up" is somewhat of a fuzzy concept
> here, since the brokers do not participate in the metadata consensus.  I
> think ideally we would want to define it in terms of time ("the broker has
> all the updates from the last 2 minutes", for example.)  We should spell
> this out better in the KIP.
>
> >
> > In section "Start the controller quorum nodes", you mention "Once it has
> > taken over the /controller node, the active controller will proceed to
> load
> > the full state of ZooKeeper.  It will write out this information to the
> > quorum's metadata storage.  After this point, the metadata quorum will be
> > the metadata store of record, rather than the data in ZooKeeper." During
> > this migration do should we expect to have a small period controller
> > unavailability while the controller replicas this state to all of the
> raft
> > nodes in the controller quorum and we buffer new controller API requests?
>
> Yes, the controller would be unavailable during this time.  I don't think
> this will be that different from the current period of unavailability when
> a new controller starts up and needs to load the full state from ZK.  The
> main difference is that in this period, we'd have to write to the
> controller quorum rather than just to memory.  But we believe this should
> be pretty fast.
>
> regards,
> Colin
>
> >
> > Thanks!
> > -Jose
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> Thanks Colin for the detail KIP. I have a few comments and questions.
> 
> In the KIP's Motivation and Overview you mentioned the LeaderAndIsr and
> UpdateMetadata RPC. For example, "updates which the controller pushes, such
> as LeaderAndIsr and UpdateMetadata messages". Is your thinking that we will
> use MetadataFetch as a replacement to just UpdateMetadata only and add
> topic configuration in this state?
> 

Hi Jose,

Thanks for taking a look.

The goal is for MetadataFetchRequest to replace both LeaderAndIsrRequest and UpdateMetadataRequest.  Topic configurations would be fetched along with the other metadata.

> In the section "Broker Metadata Management", you mention "Just like with a
> fetch request, the broker will track the offset of the last updates it
> fetched". To keep the log consistent Raft requires that the followers keep
> all of the log entries (term/epoch and offset) that are after the
> highwatermark. Any log entry before the highwatermark can be
> compacted/snapshot. Do we expect the MetadataFetch API to only return log
> entries up to the highwatermark?  Unlike the Raft replication API which
> will replicate/fetch log entries after the highwatermark for consensus?

Good question.  Clearly, we shouldn't expose metadata updates to the brokers until they've been stored on a majority of the Raft nodes.  The most obvious way to do that, like you mentioned, is to have the brokers only fetch up to the HWM, but not beyond.  There might be a more clever way to do it by fetching the data, but not having the brokers act on it until the HWM advances.  I'm not sure if that's worth it or not.  We'll discuss this more in a separate KIP that just discusses just Raft.

> 
> In section "Broker Metadata Management", you mention "the controller will
> send a full metadata image rather than a series of deltas". This KIP
> doesn't go into the set of operations that need to be supported on top of
> Raft but it would be interested if this "full metadata image" could be
> express also as deltas. For example, assuming we are replicating a map this
> "full metadata image" could be a sequence of "put" operations (znode create
> to borrow ZK semantics).

The full image can definitely be expressed as a sum of deltas.  At some point, the number of deltas will get large enough that sending a full image is better, though.  One question that we're still thinking about is how much of this can be shared with generic Kafka log code, and how much should be different.

> 
> In section "Broker Metadata Management", you mention "This request will
> double as a heartbeat, letting the controller know that the broker is
> alive". In section "Broker State Machine", you mention "The MetadataFetch
> API serves as this registration mechanism". Does this mean that the
> MetadataFetch Request will optionally include broker configuration
> information?

I was originally thinking that the MetadataFetchRequest should include broker configuration information.  Thinking about this more, maybe we should just have a special registration RPC that contains that information, to avoid sending it over the wire all the time.

> Does this also mean that MetadataFetch request will result in
> a "write"/AppendEntries through the Raft replication protocol before you
> can send the associated MetadataFetch Response?

I think we should require the broker to be out of the Offline state before allowing it to fetch metadata, yes.  So the separate registration RPC should have completed first.

> 
> In section "Broker State", you mention that a broker can transition to
> online after it is caught with the metadata. What do you mean by this?
> Metadata is always changing. How does the broker know that it is caught up
> since it doesn't participate in the consensus or the advancement of the
> highwatermark?

That's a good point.  Being "caught up" is somewhat of a fuzzy concept here, since the brokers do not participate in the metadata consensus.  I think ideally we would want to define it in terms of time ("the broker has all the updates from the last 2 minutes", for example.)  We should spell this out better in the KIP.

> 
> In section "Start the controller quorum nodes", you mention "Once it has
> taken over the /controller node, the active controller will proceed to load
> the full state of ZooKeeper.  It will write out this information to the
> quorum's metadata storage.  After this point, the metadata quorum will be
> the metadata store of record, rather than the data in ZooKeeper." During
> this migration do should we expect to have a small period controller
> unavailability while the controller replicas this state to all of the raft
> nodes in the controller quorum and we buffer new controller API requests?

Yes, the controller would be unavailable during this time.  I don't think this will be that different from the current period of unavailability when a new controller starts up and needs to load the full state from ZK.  The main difference is that in this period, we'd have to write to the controller quorum rather than just to memory.  But we believe this should be pretty fast.

regards,
Colin

> 
> Thanks!
> -Jose
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Jose Armando Garcia Sancio <js...@confluent.io>.
Thanks Colin for the detail KIP. I have a few comments and questions.

In the KIP's Motivation and Overview you mentioned the LeaderAndIsr and
UpdateMetadata RPC. For example, "updates which the controller pushes, such
as LeaderAndIsr and UpdateMetadata messages". Is your thinking that we will
use MetadataFetch as a replacement to just UpdateMetadata only and add
topic configuration in this state?

In the section "Broker Metadata Management", you mention "Just like with a
fetch request, the broker will track the offset of the last updates it
fetched". To keep the log consistent Raft requires that the followers keep
all of the log entries (term/epoch and offset) that are after the
highwatermark. Any log entry before the highwatermark can be
compacted/snapshot. Do we expect the MetadataFetch API to only return log
entries up to the highwatermark?  Unlike the Raft replication API which
will replicate/fetch log entries after the highwatermark for consensus?

In section "Broker Metadata Management", you mention "the controller will
send a full metadata image rather than a series of deltas". This KIP
doesn't go into the set of operations that need to be supported on top of
Raft but it would be interested if this "full metadata image" could be
express also as deltas. For example, assuming we are replicating a map this
"full metadata image" could be a sequence of "put" operations (znode create
to borrow ZK semantics).

In section "Broker Metadata Management", you mention "This request will
double as a heartbeat, letting the controller know that the broker is
alive". In section "Broker State Machine", you mention "The MetadataFetch
API serves as this registration mechanism". Does this mean that the
MetadataFetch Request will optionally include broker configuration
information? Does this also mean that MetadataFetch request will result in
a "write"/AppendEntries through the Raft replication protocol before you
can send the associated MetadataFetch Response?

In section "Broker State", you mention that a broker can transition to
online after it is caught with the metadata. What do you mean by this?
Metadata is always changing. How does the broker know that it is caught up
since it doesn't participate in the consensus or the advancement of the
highwatermark?

In section "Start the controller quorum nodes", you mention "Once it has
taken over the /controller node, the active controller will proceed to load
the full state of ZooKeeper.  It will write out this information to the
quorum's metadata storage.  After this point, the metadata quorum will be
the metadata store of record, rather than the data in ZooKeeper." During
this migration do should we expect to have a small period controller
unavailability while the controller replicas this state to all of the raft
nodes in the controller quorum and we buffer new controller API requests?

Thanks!
-Jose

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

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

Good idea.  I added some of this discussion to the KIP-- in particular, more about controller failover.

cheers,
Colin

On Fri, Aug 2, 2019, at 13:28, Ryanne Dolan wrote:
> Thanks Colin, that helps. Can we add some of this to the KIP?
> 
> Ryanne
> 
> On Fri, Aug 2, 2019 at 12:23 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > On Fri, Aug 2, 2019, at 07:50, Ryanne Dolan wrote:
> > > Thanks Colin, interesting KIP.
> > >
> > > I'm concerned that the KIP does not actually address its stated
> > > motivations. In particular, "Simpler Deployment and Configuration" are
> > not
> > > really achieved, given that: 1) the proposal still requires quorums (now
> > of
> > > controllers, instead of ZK nodes), with the same restrictions as ZK, e.g.
> > > at least three controllers and only an odd number of controllers, neither
> > > of which is easy to manage; 2) the proposal still requires separate
> > > processes with separate configuration (and indeed, no less configuration
> > > than ZK requires, namely a port to listen on); 3) configuration of
> > brokers
> > > is not simplified, as they still require a list of servers to contact
> > (now
> > > coordinators instead of ZK nodes). Is there any improvement to
> > > configuration and deployment I'm overlooking?
> >
> > Hi Ryanne,
> >
> > Thanks for taking a look.
> >
> > The difficulty in configuring and deploying ZooKeeper is not really in
> > configuring a port number, or even really in running a second JVM.  If that
> > were the main difficulty, then running ZK would definitely be pretty simple.
> >
> > The difficulty is that ZooKeeper is an entirely separate distributed
> > system with entirely separate configuration for things like security,
> > network setup, data directories, etc.  You also have separate systems for
> > management, metrics, and so on.  Learning how to configure security or
> > metrics in Kafka doesn't really help you with setting up the corresponding
> > features in ZK.  You have to start from scratch.  That is what we are
> > trying to avoid here.
> >
> > > Second, single-broker clusters are mentioned as a motivation, but it is
> > > unclear how this change would move in that direction. Seems Raft requires
> > > three nodes, so perhaps the minimum number of hosts would be three?
> >
> > Just like with ZooKeeper, you can run Raft on a single node.  Needless to
> > say, you don't have any tolerance against single-node failures when running
> > with a single node.
> >
> > >
> > > Third, "discrepancies between the controller state and the zookeeper
> > state"
> > > are mentioned as a problem, and I understand that controllers coordinate
> > > amongst themselves rather than via zookeeper, but I'm not sure there is a
> > > functional difference? It seems controllers can still disagree amongst
> > > themselves for periods of time, with the same consequences as disagreeing
> > > with ZK.
> >
> > Members of a Raft quorum cannot disagree with each other.  This is similar
> > to how ZooKeeper's "ZAB" protocol works.  There's more information in the
> > Raft paper: https://raft.github.io/raft.pdf
> >
> > >
> > > Finally,  you say "there is no generic way for the controller to follow
> > the
> > > ZooKeeper event log." I'm unsure this is a problem. Having a log is
> > > certainly powerful for consumers, but how would a controller use this log
> > > to do anything it can't without it? It seems only the latest compacted
> > > state is ever used, and there is nothing to undo or replay from the log.
> > > What future capabilities are you envisioning we would gain from carrying
> > > around log history?
> >
> > There are many advantages to treating metadata as a log.  Because the
> > controllers will now all track the latest state, controller failover will
> > not require a lengthy reloading period where we transfer all the state to
> > the new controller.  Because we always send deltas over the wire and not
> > full states, brokers can catch up with the latest state faster, and use
> > less bandwidth to do so.  It will even be possible for the brokers to cache
> > this state locally in a file on disk, so that broker startup can be much
> > faster.  All of these are important to scaling Kafka in the future.
> >  Treating metadata as a log avoids a lot of the complex failure corner
> > cases we have seen where a broker misses a single update sent from the
> > controller, but gets subsequent updates.
> >
> > best,
> > Colin
> >
> >
> > >
> > > Ryanne
> > >
> > >
> > > On Thu, Aug 1, 2019, 4:05 PM Colin McCabe <cm...@apache.org> wrote:
> > >
> > > > Hi all,
> > > >
> > > > I've written a KIP about removing ZooKeeper from Kafka.  Please take a
> > > > look and let me know what you think:
> > > >
> > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> > > >
> > > > cheers,
> > > > Colin
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ryanne Dolan <ry...@gmail.com>.
Thanks Colin, that helps. Can we add some of this to the KIP?

Ryanne

On Fri, Aug 2, 2019 at 12:23 PM Colin McCabe <cm...@apache.org> wrote:

> On Fri, Aug 2, 2019, at 07:50, Ryanne Dolan wrote:
> > Thanks Colin, interesting KIP.
> >
> > I'm concerned that the KIP does not actually address its stated
> > motivations. In particular, "Simpler Deployment and Configuration" are
> not
> > really achieved, given that: 1) the proposal still requires quorums (now
> of
> > controllers, instead of ZK nodes), with the same restrictions as ZK, e.g.
> > at least three controllers and only an odd number of controllers, neither
> > of which is easy to manage; 2) the proposal still requires separate
> > processes with separate configuration (and indeed, no less configuration
> > than ZK requires, namely a port to listen on); 3) configuration of
> brokers
> > is not simplified, as they still require a list of servers to contact
> (now
> > coordinators instead of ZK nodes). Is there any improvement to
> > configuration and deployment I'm overlooking?
>
> Hi Ryanne,
>
> Thanks for taking a look.
>
> The difficulty in configuring and deploying ZooKeeper is not really in
> configuring a port number, or even really in running a second JVM.  If that
> were the main difficulty, then running ZK would definitely be pretty simple.
>
> The difficulty is that ZooKeeper is an entirely separate distributed
> system with entirely separate configuration for things like security,
> network setup, data directories, etc.  You also have separate systems for
> management, metrics, and so on.  Learning how to configure security or
> metrics in Kafka doesn't really help you with setting up the corresponding
> features in ZK.  You have to start from scratch.  That is what we are
> trying to avoid here.
>
> > Second, single-broker clusters are mentioned as a motivation, but it is
> > unclear how this change would move in that direction. Seems Raft requires
> > three nodes, so perhaps the minimum number of hosts would be three?
>
> Just like with ZooKeeper, you can run Raft on a single node.  Needless to
> say, you don't have any tolerance against single-node failures when running
> with a single node.
>
> >
> > Third, "discrepancies between the controller state and the zookeeper
> state"
> > are mentioned as a problem, and I understand that controllers coordinate
> > amongst themselves rather than via zookeeper, but I'm not sure there is a
> > functional difference? It seems controllers can still disagree amongst
> > themselves for periods of time, with the same consequences as disagreeing
> > with ZK.
>
> Members of a Raft quorum cannot disagree with each other.  This is similar
> to how ZooKeeper's "ZAB" protocol works.  There's more information in the
> Raft paper: https://raft.github.io/raft.pdf
>
> >
> > Finally,  you say "there is no generic way for the controller to follow
> the
> > ZooKeeper event log." I'm unsure this is a problem. Having a log is
> > certainly powerful for consumers, but how would a controller use this log
> > to do anything it can't without it? It seems only the latest compacted
> > state is ever used, and there is nothing to undo or replay from the log.
> > What future capabilities are you envisioning we would gain from carrying
> > around log history?
>
> There are many advantages to treating metadata as a log.  Because the
> controllers will now all track the latest state, controller failover will
> not require a lengthy reloading period where we transfer all the state to
> the new controller.  Because we always send deltas over the wire and not
> full states, brokers can catch up with the latest state faster, and use
> less bandwidth to do so.  It will even be possible for the brokers to cache
> this state locally in a file on disk, so that broker startup can be much
> faster.  All of these are important to scaling Kafka in the future.
>  Treating metadata as a log avoids a lot of the complex failure corner
> cases we have seen where a broker misses a single update sent from the
> controller, but gets subsequent updates.
>
> best,
> Colin
>
>
> >
> > Ryanne
> >
> >
> > On Thu, Aug 1, 2019, 4:05 PM Colin McCabe <cm...@apache.org> wrote:
> >
> > > Hi all,
> > >
> > > I've written a KIP about removing ZooKeeper from Kafka.  Please take a
> > > look and let me know what you think:
> > >
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> > >
> > > cheers,
> > > Colin
> > >
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Colin McCabe <cm...@apache.org>.
On Fri, Aug 2, 2019, at 07:50, Ryanne Dolan wrote:
> Thanks Colin, interesting KIP.
> 
> I'm concerned that the KIP does not actually address its stated
> motivations. In particular, "Simpler Deployment and Configuration" are not
> really achieved, given that: 1) the proposal still requires quorums (now of
> controllers, instead of ZK nodes), with the same restrictions as ZK, e.g.
> at least three controllers and only an odd number of controllers, neither
> of which is easy to manage; 2) the proposal still requires separate
> processes with separate configuration (and indeed, no less configuration
> than ZK requires, namely a port to listen on); 3) configuration of brokers
> is not simplified, as they still require a list of servers to contact (now
> coordinators instead of ZK nodes). Is there any improvement to
> configuration and deployment I'm overlooking?

Hi Ryanne,

Thanks for taking a look.

The difficulty in configuring and deploying ZooKeeper is not really in configuring a port number, or even really in running a second JVM.  If that were the main difficulty, then running ZK would definitely be pretty simple.

The difficulty is that ZooKeeper is an entirely separate distributed system with entirely separate configuration for things like security, network setup, data directories, etc.  You also have separate systems for management, metrics, and so on.  Learning how to configure security or metrics in Kafka doesn't really help you with setting up the corresponding features in ZK.  You have to start from scratch.  That is what we are trying to avoid here.

> Second, single-broker clusters are mentioned as a motivation, but it is
> unclear how this change would move in that direction. Seems Raft requires
> three nodes, so perhaps the minimum number of hosts would be three?

Just like with ZooKeeper, you can run Raft on a single node.  Needless to say, you don't have any tolerance against single-node failures when running with a single node.

> 
> Third, "discrepancies between the controller state and the zookeeper state"
> are mentioned as a problem, and I understand that controllers coordinate
> amongst themselves rather than via zookeeper, but I'm not sure there is a
> functional difference? It seems controllers can still disagree amongst
> themselves for periods of time, with the same consequences as disagreeing
> with ZK.

Members of a Raft quorum cannot disagree with each other.  This is similar to how ZooKeeper's "ZAB" protocol works.  There's more information in the Raft paper: https://raft.github.io/raft.pdf

> 
> Finally,  you say "there is no generic way for the controller to follow the
> ZooKeeper event log." I'm unsure this is a problem. Having a log is
> certainly powerful for consumers, but how would a controller use this log
> to do anything it can't without it? It seems only the latest compacted
> state is ever used, and there is nothing to undo or replay from the log.
> What future capabilities are you envisioning we would gain from carrying
> around log history?

There are many advantages to treating metadata as a log.  Because the controllers will now all track the latest state, controller failover will not require a lengthy reloading period where we transfer all the state to the new controller.  Because we always send deltas over the wire and not full states, brokers can catch up with the latest state faster, and use less bandwidth to do so.  It will even be possible for the brokers to cache this state locally in a file on disk, so that broker startup can be much faster.  All of these are important to scaling Kafka in the future. 
 Treating metadata as a log avoids a lot of the complex failure corner cases we have seen where a broker misses a single update sent from the controller, but gets subsequent updates.

best,
Colin


> 
> Ryanne
> 
> 
> On Thu, Aug 1, 2019, 4:05 PM Colin McCabe <cm...@apache.org> wrote:
> 
> > Hi all,
> >
> > I've written a KIP about removing ZooKeeper from Kafka.  Please take a
> > look and let me know what you think:
> >
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> >
> > cheers,
> > Colin
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Ryanne Dolan <ry...@gmail.com>.
Thanks Colin, interesting KIP.

I'm concerned that the KIP does not actually address its stated
motivations. In particular, "Simpler Deployment and Configuration" are not
really achieved, given that: 1) the proposal still requires quorums (now of
controllers, instead of ZK nodes), with the same restrictions as ZK, e.g.
at least three controllers and only an odd number of controllers, neither
of which is easy to manage; 2) the proposal still requires separate
processes with separate configuration (and indeed, no less configuration
than ZK requires, namely a port to listen on); 3) configuration of brokers
is not simplified, as they still require a list of servers to contact (now
coordinators instead of ZK nodes). Is there any improvement to
configuration and deployment I'm overlooking?

Second, single-broker clusters are mentioned as a motivation, but it is
unclear how this change would move in that direction. Seems Raft requires
three nodes, so perhaps the minimum number of hosts would be three?

Third, "discrepancies between the controller state and the zookeeper state"
are mentioned as a problem, and I understand that controllers coordinate
amongst themselves rather than via zookeeper, but I'm not sure there is a
functional difference? It seems controllers can still disagree amongst
themselves for periods of time, with the same consequences as disagreeing
with ZK.

Finally,  you say "there is no generic way for the controller to follow the
ZooKeeper event log." I'm unsure this is a problem. Having a log is
certainly powerful for consumers, but how would a controller use this log
to do anything it can't without it? It seems only the latest compacted
state is ever used, and there is nothing to undo or replay from the log.
What future capabilities are you envisioning we would gain from carrying
around log history?

Ryanne


On Thu, Aug 1, 2019, 4:05 PM Colin McCabe <cm...@apache.org> wrote:

> Hi all,
>
> I've written a KIP about removing ZooKeeper from Kafka.  Please take a
> look and let me know what you think:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
>
> cheers,
> Colin
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

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

Thanks for the heads up.  This should be fixed-- give it another try.

best,
Colin


On Thu, Aug 1, 2019, at 14:15, Harsha wrote:
> Hi Colin,
>              Looks like KIP is missing the images , links are broken.
> Thanks,
> Harsha
> 
> On Thu, Aug 1, 2019, at 2:05 PM, Colin McCabe wrote:
> > Hi all,
> > 
> > I've written a KIP about removing ZooKeeper from Kafka.  Please take a 
> > look and let me know what you think:
> > 
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> > 
> > cheers,
> > Colin
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Harsha <ka...@harsha.io>.
Hi Colin,
             Looks like KIP is missing the images , links are broken.
Thanks,
Harsha

On Thu, Aug 1, 2019, at 2:05 PM, Colin McCabe wrote:
> Hi all,
> 
> I've written a KIP about removing ZooKeeper from Kafka.  Please take a 
> look and let me know what you think:
> 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> 
> cheers,
> Colin
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Viktor Somogyi-Vass <vi...@gmail.com>.
Hi Elmahdi,

I've added the JIRA to the KIP (and also below) where you can track the
progress (but more subtask will come as the current ones don't represent
the full work to be done).
https://issues.apache.org/jira/browse/KAFKA-9119

Viktor

On Tue, Nov 5, 2019 at 5:15 PM Elmahdi FRID <fr...@gmail.com> wrote:

> Hello Folks any status abbout this kip and it's possible to test this use
> case ?
>
> On 2019/08/01 21:04:46, "Colin McCabe" <cm...@apache.org> wrote:
> > Hi all,
> >
> > I've written a KIP about removing ZooKeeper from Kafka.  Please take a
> look and let me know what you think:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> >
> > cheers,
> > Colin
> >
>

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

Posted by Elmahdi FRID <fr...@gmail.com>.
Hello Folks any status abbout this kip and it's possible to test this use case ?

On 2019/08/01 21:04:46, "Colin McCabe" <cm...@apache.org> wrote: 
> Hi all,
> 
> I've written a KIP about removing ZooKeeper from Kafka.  Please take a look and let me know what you think:
> 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
> 
> cheers,
> Colin
>