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 <cmcc...@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 <cmcc...@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 <cmcc...@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 < > rndg...@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 < > cmcc...@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 < > > > cmcc...@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 < > > > > > cmcc...@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 < > > > > > > >> > > > viktorsomo...@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 < > > > > > > >> > > > reluctanthero...@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 < > > > > > > >> ism...@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 < > > > > > > >> > > > reluctanthero...@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 < > > > > > > >> > > > cmcc...@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 > > > > > > >> > > > > > > > > >> > > > > > > > >> > > > > > > > > > > > > > > > > > > > > > > > > > > > >