Re: [DISCUSS] Kafka 3.0

2021-02-22 Thread Guozhang Wang
+1 on getting to 3.0 for the June release this year too.

Guozhang

On Mon, Feb 22, 2021 at 6:54 PM Matthias J. Sax  wrote:

> To move this forward, I took the liberty to create a PR to bump the
> version to 3.0.0-SNAPSHOT
>
> https://github.com/apache/kafka/pull/10186
>
> Please let us know if there are any concerns.
>
>
> -Matthias
>
> On 2/16/21 5:18 PM, Ismael Juma wrote:
> > I'm +1 on 3.0 for the mid year release.
> >
> > On Tue, Feb 16, 2021 at 5:08 PM Matthias J. Sax 
> wrote:
> >
> >> Hi,
> >>
> >> given that we passed 2.8 feature freeze, I wanted to restart this
> >> thread. Currently, `trunk` is at `2.9.0-SNAPSHOT` and I am wondering if
> >> the decision for the 3.0 release is final and if we should bump the
> >> version number?
> >>
> >> I am asking particularly because there a many Jiras with a 3.0 target
> >> release version for breaking changes and we should ensure that we have
> >> enough time to work on those tickets. -- As long as we don't agree that
> >> the next release will indeed be 3.0, those tickets are effectively
> >> blocked/pending.
> >>
> >> Thoughts?
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 10/15/20 4:28 PM, Matthias J. Sax wrote:
> >>> Thanks for clarifying Colin. Works for me. Overall, 3.0 should be
> guided
> >>> by the ZK removal progress and if we are not there yet, it's better to
> >>> have a 2.8 first.
> >>>
> >>>
> >>> -Matthias
> >>>
> >>>
> >>> On 10/15/20 2:41 PM, Colin McCabe wrote:
>  Hi all,
> 
>  Just to follow up on this... since we're not quite ready for 3.0 yet,
> >> it's probably best if we release a 2.8 next, and then go to 3.0 after
> >> that.  Sorry for any confusion.
> 
>  best,
>  Colin
> 
> 
>  On Mon, Jul 20, 2020, at 12:52, Matthias J. Sax wrote:
> > Did we reach any conclusion on the subject?
> >
> > It seems we are aiming for 2.7 after 2.6 and plan the major version
> >> bump
> > to 3.0 after 2.7 (assuming we make progress on ZK removal as
> planned?)
> >
> >
> > -Matthias
> >
> >
> > On 5/18/20 1:11 PM, Boyang Chen wrote:
> >> One more thing I would like to see deprecated (hopefully no one
> >> mentioned
> >> before) is the zk based consumer offset support.
> >>
> >> On Mon, May 11, 2020 at 2:15 PM Colin McCabe 
> >> wrote:
> >>
> >>> Hi Michael,
> >>>
> >>> It would be better to discuss the background behind KIP-500 in a
> >> separate
> >>> thread, since this thread is about the Kafka 3.0 release.  As
> others
> >> have
> >>> said, your questions are answered in the KIP.  For example, "what
> is
> >> the
> >>> actual goal?" is addressed in the motivation section.
> >>>
> >>> I agree that Kafka's usage of Apache ZooKeeper could be optimized.
> >> But
> >>> there are fundamental limitations to this approach compared to
> >> storing our
> >>> metadata internally.  For example, having to contact a remote
> server
> >> to
> >>> reload all your metadata on a controller failover simply doesn't
> >> scale past
> >>> a certain point.
> >>>
> >>> Apache Curator is a nice API, and if we were starting again today
> we
> >> would
> >>> certainly consider using it.  But it doesn't allow us to do
> anything
> >> more
> >>> efficiently than ZooKeeper could already do it.
> >>>
> >>> Finally, Kafka's core competence is logs.  While our replication
> >> protocol
> >>> is not Raft, it shares many similarities with that protocol.  So I
> >> think
> >>> it's a bit unfair to say that it is "catastrophic hubris" to
> believe
> >> we can
> >>> implement the protocol.
> >>>
> >>> best,
> >>> Colin
> >>>
> >>>
> >>> On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
>  Yes, I've read the KIP.  But all it really says to me is "we have
> >> never
>  gotten around to using ZooKeeper properly."  To the extent that
> any
> >> of
> >>> the
>  distributed-state-maintenance problems discussed in "Metadata as
> an
> >> Event
>  Log" can be solved — and some of them intrinsically can't, because
> >> CAP
>  theorem — most of them are already implemented very effectively in
> >>> Curator
>  recipes.  (For instance, Curator's Tree Cache
>  https://curator.apache.org/curator-recipes/tree-cache.html is a
> >> good
> >>> fit to
>  some of the state-maintenance needs.)
> 
>  Kafka does have some usage patterns that don't map neatly onto
> >> existing
>  Curator recipes.  For instance, neither LeaderSelector nor
> >> LeaderLatch
>  implements leader preference in the way that the existing Kafka
> >> partition
>  leadership election procedure does.  But why not handle that by
> >> improving
>  and extending Curator?  That way, other Curator users benefit, and
> >> we get
>  additional highly experienced reviewers' eyes on the distributed
>  

Re: [DISCUSS] Kafka 3.0

2021-02-22 Thread Matthias J. Sax
To move this forward, I took the liberty to create a PR to bump the
version to 3.0.0-SNAPSHOT

https://github.com/apache/kafka/pull/10186

Please let us know if there are any concerns.


-Matthias

On 2/16/21 5:18 PM, Ismael Juma wrote:
> I'm +1 on 3.0 for the mid year release.
> 
> On Tue, Feb 16, 2021 at 5:08 PM Matthias J. Sax  wrote:
> 
>> Hi,
>>
>> given that we passed 2.8 feature freeze, I wanted to restart this
>> thread. Currently, `trunk` is at `2.9.0-SNAPSHOT` and I am wondering if
>> the decision for the 3.0 release is final and if we should bump the
>> version number?
>>
>> I am asking particularly because there a many Jiras with a 3.0 target
>> release version for breaking changes and we should ensure that we have
>> enough time to work on those tickets. -- As long as we don't agree that
>> the next release will indeed be 3.0, those tickets are effectively
>> blocked/pending.
>>
>> Thoughts?
>>
>>
>> -Matthias
>>
>>
>> On 10/15/20 4:28 PM, Matthias J. Sax wrote:
>>> Thanks for clarifying Colin. Works for me. Overall, 3.0 should be guided
>>> by the ZK removal progress and if we are not there yet, it's better to
>>> have a 2.8 first.
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 10/15/20 2:41 PM, Colin McCabe wrote:
 Hi all,

 Just to follow up on this... since we're not quite ready for 3.0 yet,
>> it's probably best if we release a 2.8 next, and then go to 3.0 after
>> that.  Sorry for any confusion.

 best,
 Colin


 On Mon, Jul 20, 2020, at 12:52, Matthias J. Sax wrote:
> Did we reach any conclusion on the subject?
>
> It seems we are aiming for 2.7 after 2.6 and plan the major version
>> bump
> to 3.0 after 2.7 (assuming we make progress on ZK removal as planned?)
>
>
> -Matthias
>
>
> On 5/18/20 1:11 PM, Boyang Chen wrote:
>> One more thing I would like to see deprecated (hopefully no one
>> mentioned
>> before) is the zk based consumer offset support.
>>
>> On Mon, May 11, 2020 at 2:15 PM Colin McCabe 
>> wrote:
>>
>>> Hi Michael,
>>>
>>> It would be better to discuss the background behind KIP-500 in a
>> separate
>>> thread, since this thread is about the Kafka 3.0 release.  As others
>> have
>>> said, your questions are answered in the KIP.  For example, "what is
>> the
>>> actual goal?" is addressed in the motivation section.
>>>
>>> I agree that Kafka's usage of Apache ZooKeeper could be optimized.
>> But
>>> there are fundamental limitations to this approach compared to
>> storing our
>>> metadata internally.  For example, having to contact a remote server
>> to
>>> reload all your metadata on a controller failover simply doesn't
>> scale past
>>> a certain point.
>>>
>>> Apache Curator is a nice API, and if we were starting again today we
>> would
>>> certainly consider using it.  But it doesn't allow us to do anything
>> more
>>> efficiently than ZooKeeper could already do it.
>>>
>>> Finally, Kafka's core competence is logs.  While our replication
>> protocol
>>> is not Raft, it shares many similarities with that protocol.  So I
>> think
>>> it's a bit unfair to say that it is "catastrophic hubris" to believe
>> we can
>>> implement the protocol.
>>>
>>> best,
>>> Colin
>>>
>>>
>>> On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
 Yes, I've read the KIP.  But all it really says to me is "we have
>> never
 gotten around to using ZooKeeper properly."  To the extent that any
>> of
>>> the
 distributed-state-maintenance problems discussed in "Metadata as an
>> Event
 Log" can be solved — and some of them intrinsically can't, because
>> CAP
 theorem — most of them are already implemented very effectively in
>>> Curator
 recipes.  (For instance, Curator's Tree Cache
 https://curator.apache.org/curator-recipes/tree-cache.html is a
>> good
>>> fit to
 some of the state-maintenance needs.)

 Kafka does have some usage patterns that don't map neatly onto
>> existing
 Curator recipes.  For instance, neither LeaderSelector nor
>> LeaderLatch
 implements leader preference in the way that the existing Kafka
>> partition
 leadership election procedure does.  But why not handle that by
>> improving
 and extending Curator?  That way, other Curator users benefit, and
>> we get
 additional highly experienced reviewers' eyes on the distributed
 algorithms, which are very very tricky to get right.


 On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
>>> wrote:

> Hi Michael.  This is discussed in the KIP.
>
>
>
>>>
>> 

Re: [DISCUSS] Kafka 3.0

2021-02-16 Thread Ismael Juma
I'm +1 on 3.0 for the mid year release.

On Tue, Feb 16, 2021 at 5:08 PM Matthias J. Sax  wrote:

> Hi,
>
> given that we passed 2.8 feature freeze, I wanted to restart this
> thread. Currently, `trunk` is at `2.9.0-SNAPSHOT` and I am wondering if
> the decision for the 3.0 release is final and if we should bump the
> version number?
>
> I am asking particularly because there a many Jiras with a 3.0 target
> release version for breaking changes and we should ensure that we have
> enough time to work on those tickets. -- As long as we don't agree that
> the next release will indeed be 3.0, those tickets are effectively
> blocked/pending.
>
> Thoughts?
>
>
> -Matthias
>
>
> On 10/15/20 4:28 PM, Matthias J. Sax wrote:
> > Thanks for clarifying Colin. Works for me. Overall, 3.0 should be guided
> > by the ZK removal progress and if we are not there yet, it's better to
> > have a 2.8 first.
> >
> >
> > -Matthias
> >
> >
> > On 10/15/20 2:41 PM, Colin McCabe wrote:
> >> Hi all,
> >>
> >> Just to follow up on this... since we're not quite ready for 3.0 yet,
> it's probably best if we release a 2.8 next, and then go to 3.0 after
> that.  Sorry for any confusion.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> On Mon, Jul 20, 2020, at 12:52, Matthias J. Sax wrote:
> >>> Did we reach any conclusion on the subject?
> >>>
> >>> It seems we are aiming for 2.7 after 2.6 and plan the major version
> bump
> >>> to 3.0 after 2.7 (assuming we make progress on ZK removal as planned?)
> >>>
> >>>
> >>> -Matthias
> >>>
> >>>
> >>> On 5/18/20 1:11 PM, Boyang Chen wrote:
>  One more thing I would like to see deprecated (hopefully no one
> mentioned
>  before) is the zk based consumer offset support.
> 
>  On Mon, May 11, 2020 at 2:15 PM Colin McCabe 
> wrote:
> 
> > Hi Michael,
> >
> > It would be better to discuss the background behind KIP-500 in a
> separate
> > thread, since this thread is about the Kafka 3.0 release.  As others
> have
> > said, your questions are answered in the KIP.  For example, "what is
> the
> > actual goal?" is addressed in the motivation section.
> >
> > I agree that Kafka's usage of Apache ZooKeeper could be optimized.
> But
> > there are fundamental limitations to this approach compared to
> storing our
> > metadata internally.  For example, having to contact a remote server
> to
> > reload all your metadata on a controller failover simply doesn't
> scale past
> > a certain point.
> >
> > Apache Curator is a nice API, and if we were starting again today we
> would
> > certainly consider using it.  But it doesn't allow us to do anything
> more
> > efficiently than ZooKeeper could already do it.
> >
> > Finally, Kafka's core competence is logs.  While our replication
> protocol
> > is not Raft, it shares many similarities with that protocol.  So I
> think
> > it's a bit unfair to say that it is "catastrophic hubris" to believe
> we can
> > implement the protocol.
> >
> > best,
> > Colin
> >
> >
> > On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
> >> Yes, I've read the KIP.  But all it really says to me is "we have
> never
> >> gotten around to using ZooKeeper properly."  To the extent that any
> of
> > the
> >> distributed-state-maintenance problems discussed in "Metadata as an
> Event
> >> Log" can be solved — and some of them intrinsically can't, because
> CAP
> >> theorem — most of them are already implemented very effectively in
> > Curator
> >> recipes.  (For instance, Curator's Tree Cache
> >> https://curator.apache.org/curator-recipes/tree-cache.html is a
> good
> > fit to
> >> some of the state-maintenance needs.)
> >>
> >> Kafka does have some usage patterns that don't map neatly onto
> existing
> >> Curator recipes.  For instance, neither LeaderSelector nor
> LeaderLatch
> >> implements leader preference in the way that the existing Kafka
> partition
> >> leadership election procedure does.  But why not handle that by
> improving
> >> and extending Curator?  That way, other Curator users benefit, and
> we get
> >> additional highly experienced reviewers' eyes on the distributed
> >> algorithms, which are very very tricky to get right.
> >>
> >>
> >> On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
> > wrote:
> >>
> >>> Hi Michael.  This is discussed in the KIP.
> >>>
> >>>
> >>>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> >>>
> >>> Ron
> >>>
>  On May 10, 2020, at 1:35 PM, Michael K. Edwards <
> > m.k.edwa...@gmail.com>
> >>> wrote:
> 
>  What is the actual goal of removing the ZooKeeper dependency?  In
> my
>  experience, if ZooKeeper is properly 

Re: [DISCUSS] Kafka 3.0

2021-02-16 Thread Matthias J. Sax
Hi,

given that we passed 2.8 feature freeze, I wanted to restart this
thread. Currently, `trunk` is at `2.9.0-SNAPSHOT` and I am wondering if
the decision for the 3.0 release is final and if we should bump the
version number?

I am asking particularly because there a many Jiras with a 3.0 target
release version for breaking changes and we should ensure that we have
enough time to work on those tickets. -- As long as we don't agree that
the next release will indeed be 3.0, those tickets are effectively
blocked/pending.

Thoughts?


-Matthias


On 10/15/20 4:28 PM, Matthias J. Sax wrote:
> Thanks for clarifying Colin. Works for me. Overall, 3.0 should be guided
> by the ZK removal progress and if we are not there yet, it's better to
> have a 2.8 first.
> 
> 
> -Matthias
> 
> 
> On 10/15/20 2:41 PM, Colin McCabe wrote:
>> Hi all,
>>
>> Just to follow up on this... since we're not quite ready for 3.0 yet, it's 
>> probably best if we release a 2.8 next, and then go to 3.0 after that.  
>> Sorry for any confusion.
>>
>> best,
>> Colin
>>
>>
>> On Mon, Jul 20, 2020, at 12:52, Matthias J. Sax wrote:
>>> Did we reach any conclusion on the subject?
>>>
>>> It seems we are aiming for 2.7 after 2.6 and plan the major version bump
>>> to 3.0 after 2.7 (assuming we make progress on ZK removal as planned?)
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 5/18/20 1:11 PM, Boyang Chen wrote:
 One more thing I would like to see deprecated (hopefully no one mentioned
 before) is the zk based consumer offset support.

 On Mon, May 11, 2020 at 2:15 PM Colin McCabe  wrote:

> Hi Michael,
>
> It would be better to discuss the background behind KIP-500 in a separate
> thread, since this thread is about the Kafka 3.0 release.  As others have
> said, your questions are answered in the KIP.  For example, "what is the
> actual goal?" is addressed in the motivation section.
>
> I agree that Kafka's usage of Apache ZooKeeper could be optimized.  But
> there are fundamental limitations to this approach compared to storing our
> metadata internally.  For example, having to contact a remote server to
> reload all your metadata on a controller failover simply doesn't scale 
> past
> a certain point.
>
> Apache Curator is a nice API, and if we were starting again today we would
> certainly consider using it.  But it doesn't allow us to do anything more
> efficiently than ZooKeeper could already do it.
>
> Finally, Kafka's core competence is logs.  While our replication protocol
> is not Raft, it shares many similarities with that protocol.  So I think
> it's a bit unfair to say that it is "catastrophic hubris" to believe we 
> can
> implement the protocol.
>
> best,
> Colin
>
>
> On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
>> Yes, I've read the KIP.  But all it really says to me is "we have never
>> gotten around to using ZooKeeper properly."  To the extent that any of
> the
>> distributed-state-maintenance problems discussed in "Metadata as an Event
>> Log" can be solved — and some of them intrinsically can't, because CAP
>> theorem — most of them are already implemented very effectively in
> Curator
>> recipes.  (For instance, Curator's Tree Cache
>> https://curator.apache.org/curator-recipes/tree-cache.html is a good
> fit to
>> some of the state-maintenance needs.)
>>
>> Kafka does have some usage patterns that don't map neatly onto existing
>> Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
>> implements leader preference in the way that the existing Kafka partition
>> leadership election procedure does.  But why not handle that by improving
>> and extending Curator?  That way, other Curator users benefit, and we get
>> additional highly experienced reviewers' eyes on the distributed
>> algorithms, which are very very tricky to get right.
>>
>>
>> On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
> wrote:
>>
>>> Hi Michael.  This is discussed in the KIP.
>>>
>>>
>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
>>>
>>> Ron
>>>
 On May 10, 2020, at 1:35 PM, Michael K. Edwards <
> m.k.edwa...@gmail.com>
>>> wrote:

 What is the actual goal of removing the ZooKeeper dependency?  In my
 experience, if ZooKeeper is properly provisioned and deployed, it's
>>> largely
 trouble-free.  (You do need to know how to use observers properly.)
>>> There
 are some subtleties about timeouts and leadership changes, but
> they're
 pretty small stuff.  Why go to all the trouble of building a new
 distributed-consensus system that's going to have 

Re: [DISCUSS] Kafka 3.0

2020-10-15 Thread Matthias J. Sax
Thanks for clarifying Colin. Works for me. Overall, 3.0 should be guided
by the ZK removal progress and if we are not there yet, it's better to
have a 2.8 first.


-Matthias


On 10/15/20 2:41 PM, Colin McCabe wrote:
> Hi all,
> 
> Just to follow up on this... since we're not quite ready for 3.0 yet, it's 
> probably best if we release a 2.8 next, and then go to 3.0 after that.  Sorry 
> for any confusion.
> 
> best,
> Colin
> 
> 
> On Mon, Jul 20, 2020, at 12:52, Matthias J. Sax wrote:
>> Did we reach any conclusion on the subject?
>>
>> It seems we are aiming for 2.7 after 2.6 and plan the major version bump
>> to 3.0 after 2.7 (assuming we make progress on ZK removal as planned?)
>>
>>
>> -Matthias
>>
>>
>> On 5/18/20 1:11 PM, Boyang Chen wrote:
>>> One more thing I would like to see deprecated (hopefully no one mentioned
>>> before) is the zk based consumer offset support.
>>>
>>> On Mon, May 11, 2020 at 2:15 PM Colin McCabe  wrote:
>>>
 Hi Michael,

 It would be better to discuss the background behind KIP-500 in a separate
 thread, since this thread is about the Kafka 3.0 release.  As others have
 said, your questions are answered in the KIP.  For example, "what is the
 actual goal?" is addressed in the motivation section.

 I agree that Kafka's usage of Apache ZooKeeper could be optimized.  But
 there are fundamental limitations to this approach compared to storing our
 metadata internally.  For example, having to contact a remote server to
 reload all your metadata on a controller failover simply doesn't scale past
 a certain point.

 Apache Curator is a nice API, and if we were starting again today we would
 certainly consider using it.  But it doesn't allow us to do anything more
 efficiently than ZooKeeper could already do it.

 Finally, Kafka's core competence is logs.  While our replication protocol
 is not Raft, it shares many similarities with that protocol.  So I think
 it's a bit unfair to say that it is "catastrophic hubris" to believe we can
 implement the protocol.

 best,
 Colin


 On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
> Yes, I've read the KIP.  But all it really says to me is "we have never
> gotten around to using ZooKeeper properly."  To the extent that any of
 the
> distributed-state-maintenance problems discussed in "Metadata as an Event
> Log" can be solved — and some of them intrinsically can't, because CAP
> theorem — most of them are already implemented very effectively in
 Curator
> recipes.  (For instance, Curator's Tree Cache
> https://curator.apache.org/curator-recipes/tree-cache.html is a good
 fit to
> some of the state-maintenance needs.)
>
> Kafka does have some usage patterns that don't map neatly onto existing
> Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> implements leader preference in the way that the existing Kafka partition
> leadership election procedure does.  But why not handle that by improving
> and extending Curator?  That way, other Curator users benefit, and we get
> additional highly experienced reviewers' eyes on the distributed
> algorithms, which are very very tricky to get right.
>
>
> On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
 wrote:
>
>> Hi Michael.  This is discussed in the KIP.
>>
>>
>>
 https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
>>
>> Ron
>>
>>> On May 10, 2020, at 1:35 PM, Michael K. Edwards <
 m.k.edwa...@gmail.com>
>> wrote:
>>>
>>> What is the actual goal of removing the ZooKeeper dependency?  In my
>>> experience, if ZooKeeper is properly provisioned and deployed, it's
>> largely
>>> trouble-free.  (You do need to know how to use observers properly.)
>> There
>>> are some subtleties about timeouts and leadership changes, but
 they're
>>> pretty small stuff.  Why go to all the trouble of building a new
>>> distributed-consensus system that's going to have catastrophic bugs
 for
>>> years to come?  It seems like such an act of hubris to me, as well
 as a
>>> massive waste of engineering effort.  What is there to be gained?
>>>
 On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
>> wrote:

 Sure, we can compile a list for Kafka Streams. But the KIP would be
 for
 3.0, so I don't think it's urgent to do it now?


 -Matthias

> On 5/8/20 3:47 PM, Colin McCabe wrote:
> Thanks, Guozhang-- sounds like a good plan.
>
> I think it would be good to have a list of deprecated streams APIs
 that
 we want to remove in 3.0.  Maybe it's easiest to do 

Re: [DISCUSS] Kafka 3.0

2020-10-15 Thread Colin McCabe
Hi all,

Just to follow up on this... since we're not quite ready for 3.0 yet, it's 
probably best if we release a 2.8 next, and then go to 3.0 after that.  Sorry 
for any confusion.

best,
Colin


On Mon, Jul 20, 2020, at 12:52, Matthias J. Sax wrote:
> Did we reach any conclusion on the subject?
> 
> It seems we are aiming for 2.7 after 2.6 and plan the major version bump
> to 3.0 after 2.7 (assuming we make progress on ZK removal as planned?)
> 
> 
> -Matthias
> 
> 
> On 5/18/20 1:11 PM, Boyang Chen wrote:
> > One more thing I would like to see deprecated (hopefully no one mentioned
> > before) is the zk based consumer offset support.
> > 
> > On Mon, May 11, 2020 at 2:15 PM Colin McCabe  wrote:
> > 
> >> Hi Michael,
> >>
> >> It would be better to discuss the background behind KIP-500 in a separate
> >> thread, since this thread is about the Kafka 3.0 release.  As others have
> >> said, your questions are answered in the KIP.  For example, "what is the
> >> actual goal?" is addressed in the motivation section.
> >>
> >> I agree that Kafka's usage of Apache ZooKeeper could be optimized.  But
> >> there are fundamental limitations to this approach compared to storing our
> >> metadata internally.  For example, having to contact a remote server to
> >> reload all your metadata on a controller failover simply doesn't scale past
> >> a certain point.
> >>
> >> Apache Curator is a nice API, and if we were starting again today we would
> >> certainly consider using it.  But it doesn't allow us to do anything more
> >> efficiently than ZooKeeper could already do it.
> >>
> >> Finally, Kafka's core competence is logs.  While our replication protocol
> >> is not Raft, it shares many similarities with that protocol.  So I think
> >> it's a bit unfair to say that it is "catastrophic hubris" to believe we can
> >> implement the protocol.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
> >>> Yes, I've read the KIP.  But all it really says to me is "we have never
> >>> gotten around to using ZooKeeper properly."  To the extent that any of
> >> the
> >>> distributed-state-maintenance problems discussed in "Metadata as an Event
> >>> Log" can be solved — and some of them intrinsically can't, because CAP
> >>> theorem — most of them are already implemented very effectively in
> >> Curator
> >>> recipes.  (For instance, Curator's Tree Cache
> >>> https://curator.apache.org/curator-recipes/tree-cache.html is a good
> >> fit to
> >>> some of the state-maintenance needs.)
> >>>
> >>> Kafka does have some usage patterns that don't map neatly onto existing
> >>> Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> >>> implements leader preference in the way that the existing Kafka partition
> >>> leadership election procedure does.  But why not handle that by improving
> >>> and extending Curator?  That way, other Curator users benefit, and we get
> >>> additional highly experienced reviewers' eyes on the distributed
> >>> algorithms, which are very very tricky to get right.
> >>>
> >>>
> >>> On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
> >> wrote:
> >>>
>  Hi Michael.  This is discussed in the KIP.
> 
> 
> 
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> 
>  Ron
> 
> > On May 10, 2020, at 1:35 PM, Michael K. Edwards <
> >> m.k.edwa...@gmail.com>
>  wrote:
> >
> > What is the actual goal of removing the ZooKeeper dependency?  In my
> > experience, if ZooKeeper is properly provisioned and deployed, it's
>  largely
> > trouble-free.  (You do need to know how to use observers properly.)
>  There
> > are some subtleties about timeouts and leadership changes, but
> >> they're
> > pretty small stuff.  Why go to all the trouble of building a new
> > distributed-consensus system that's going to have catastrophic bugs
> >> for
> > years to come?  It seems like such an act of hubris to me, as well
> >> as a
> > massive waste of engineering effort.  What is there to be gained?
> >
> >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
>  wrote:
> >>
> >> Sure, we can compile a list for Kafka Streams. But the KIP would be
> >> for
> >> 3.0, so I don't think it's urgent to do it now?
> >>
> >>
> >> -Matthias
> >>
> >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> >>> Thanks, Guozhang-- sounds like a good plan.
> >>>
> >>> I think it would be good to have a list of deprecated streams APIs
> >> that
> >> we want to remove in 3.0.  Maybe it's easiest to do that as its own
> >> KIP?
> >>>
> >>> For MirrorMaker 1, we should have a KIP to deprecate its use in
> >> 2.6 if
> >> we want to remove it in 3.0.  I don't have a good sense of how
>  practical it
> >> is to deprecate this 

Re: [DISCUSS] Kafka 3.0

2020-07-20 Thread Matthias J. Sax
Did we reach any conclusion on the subject?

It seems we are aiming for 2.7 after 2.6 and plan the major version bump
to 3.0 after 2.7 (assuming we make progress on ZK removal as planned?)


-Matthias


On 5/18/20 1:11 PM, Boyang Chen wrote:
> One more thing I would like to see deprecated (hopefully no one mentioned
> before) is the zk based consumer offset support.
> 
> On Mon, May 11, 2020 at 2:15 PM Colin McCabe  wrote:
> 
>> Hi Michael,
>>
>> It would be better to discuss the background behind KIP-500 in a separate
>> thread, since this thread is about the Kafka 3.0 release.  As others have
>> said, your questions are answered in the KIP.  For example, "what is the
>> actual goal?" is addressed in the motivation section.
>>
>> I agree that Kafka's usage of Apache ZooKeeper could be optimized.  But
>> there are fundamental limitations to this approach compared to storing our
>> metadata internally.  For example, having to contact a remote server to
>> reload all your metadata on a controller failover simply doesn't scale past
>> a certain point.
>>
>> Apache Curator is a nice API, and if we were starting again today we would
>> certainly consider using it.  But it doesn't allow us to do anything more
>> efficiently than ZooKeeper could already do it.
>>
>> Finally, Kafka's core competence is logs.  While our replication protocol
>> is not Raft, it shares many similarities with that protocol.  So I think
>> it's a bit unfair to say that it is "catastrophic hubris" to believe we can
>> implement the protocol.
>>
>> best,
>> Colin
>>
>>
>> On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
>>> Yes, I've read the KIP.  But all it really says to me is "we have never
>>> gotten around to using ZooKeeper properly."  To the extent that any of
>> the
>>> distributed-state-maintenance problems discussed in "Metadata as an Event
>>> Log" can be solved — and some of them intrinsically can't, because CAP
>>> theorem — most of them are already implemented very effectively in
>> Curator
>>> recipes.  (For instance, Curator's Tree Cache
>>> https://curator.apache.org/curator-recipes/tree-cache.html is a good
>> fit to
>>> some of the state-maintenance needs.)
>>>
>>> Kafka does have some usage patterns that don't map neatly onto existing
>>> Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
>>> implements leader preference in the way that the existing Kafka partition
>>> leadership election procedure does.  But why not handle that by improving
>>> and extending Curator?  That way, other Curator users benefit, and we get
>>> additional highly experienced reviewers' eyes on the distributed
>>> algorithms, which are very very tricky to get right.
>>>
>>>
>>> On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
>> wrote:
>>>
 Hi Michael.  This is discussed in the KIP.



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

 Ron

> On May 10, 2020, at 1:35 PM, Michael K. Edwards <
>> m.k.edwa...@gmail.com>
 wrote:
>
> What is the actual goal of removing the ZooKeeper dependency?  In my
> experience, if ZooKeeper is properly provisioned and deployed, it's
 largely
> trouble-free.  (You do need to know how to use observers properly.)
 There
> are some subtleties about timeouts and leadership changes, but
>> they're
> pretty small stuff.  Why go to all the trouble of building a new
> distributed-consensus system that's going to have catastrophic bugs
>> for
> years to come?  It seems like such an act of hubris to me, as well
>> as a
> massive waste of engineering effort.  What is there to be gained?
>
>> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
 wrote:
>>
>> Sure, we can compile a list for Kafka Streams. But the KIP would be
>> for
>> 3.0, so I don't think it's urgent to do it now?
>>
>>
>> -Matthias
>>
>>> On 5/8/20 3:47 PM, Colin McCabe wrote:
>>> Thanks, Guozhang-- sounds like a good plan.
>>>
>>> I think it would be good to have a list of deprecated streams APIs
>> that
>> we want to remove in 3.0.  Maybe it's easiest to do that as its own
>> KIP?
>>>
>>> For MirrorMaker 1, we should have a KIP to deprecate its use in
>> 2.6 if
>> we want to remove it in 3.0.  I don't have a good sense of how
 practical it
>> is to deprecate this now, so I will defer to others here.  But the
>> KIP
>> freeze for 2.6 is coming soon, so if we want to make the case, now
>> is
 the
>> time.
>>>
>>> best,
>>> Colin
>>>
>>>
 On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
 Hey folks,

 Sorry for stating that the bridge release would not break any
>> compatibility
 before, which is incorrect and confused many people.

 I think one way 

Re: [DISCUSS] Kafka 3.0

2020-05-18 Thread Boyang Chen
One more thing I would like to see deprecated (hopefully no one mentioned
before) is the zk based consumer offset support.

On Mon, May 11, 2020 at 2:15 PM Colin McCabe  wrote:

> Hi Michael,
>
> It would be better to discuss the background behind KIP-500 in a separate
> thread, since this thread is about the Kafka 3.0 release.  As others have
> said, your questions are answered in the KIP.  For example, "what is the
> actual goal?" is addressed in the motivation section.
>
> I agree that Kafka's usage of Apache ZooKeeper could be optimized.  But
> there are fundamental limitations to this approach compared to storing our
> metadata internally.  For example, having to contact a remote server to
> reload all your metadata on a controller failover simply doesn't scale past
> a certain point.
>
> Apache Curator is a nice API, and if we were starting again today we would
> certainly consider using it.  But it doesn't allow us to do anything more
> efficiently than ZooKeeper could already do it.
>
> Finally, Kafka's core competence is logs.  While our replication protocol
> is not Raft, it shares many similarities with that protocol.  So I think
> it's a bit unfair to say that it is "catastrophic hubris" to believe we can
> implement the protocol.
>
> best,
> Colin
>
>
> On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
> > Yes, I've read the KIP.  But all it really says to me is "we have never
> > gotten around to using ZooKeeper properly."  To the extent that any of
> the
> > distributed-state-maintenance problems discussed in "Metadata as an Event
> > Log" can be solved — and some of them intrinsically can't, because CAP
> > theorem — most of them are already implemented very effectively in
> Curator
> > recipes.  (For instance, Curator's Tree Cache
> > https://curator.apache.org/curator-recipes/tree-cache.html is a good
> fit to
> > some of the state-maintenance needs.)
> >
> > Kafka does have some usage patterns that don't map neatly onto existing
> > Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> > implements leader preference in the way that the existing Kafka partition
> > leadership election procedure does.  But why not handle that by improving
> > and extending Curator?  That way, other Curator users benefit, and we get
> > additional highly experienced reviewers' eyes on the distributed
> > algorithms, which are very very tricky to get right.
> >
> >
> > On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
> wrote:
> >
> > > Hi Michael.  This is discussed in the KIP.
> > >
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> > >
> > > Ron
> > >
> > > > On May 10, 2020, at 1:35 PM, Michael K. Edwards <
> m.k.edwa...@gmail.com>
> > > wrote:
> > > >
> > > > What is the actual goal of removing the ZooKeeper dependency?  In my
> > > > experience, if ZooKeeper is properly provisioned and deployed, it's
> > > largely
> > > > trouble-free.  (You do need to know how to use observers properly.)
> > > There
> > > > are some subtleties about timeouts and leadership changes, but
> they're
> > > > pretty small stuff.  Why go to all the trouble of building a new
> > > > distributed-consensus system that's going to have catastrophic bugs
> for
> > > > years to come?  It seems like such an act of hubris to me, as well
> as a
> > > > massive waste of engineering effort.  What is there to be gained?
> > > >
> > > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
> > > wrote:
> > > >>
> > > >> Sure, we can compile a list for Kafka Streams. But the KIP would be
> for
> > > >> 3.0, so I don't think it's urgent to do it now?
> > > >>
> > > >>
> > > >> -Matthias
> > > >>
> > > >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > > >>> Thanks, Guozhang-- sounds like a good plan.
> > > >>>
> > > >>> I think it would be good to have a list of deprecated streams APIs
> that
> > > >> we want to remove in 3.0.  Maybe it's easiest to do that as its own
> KIP?
> > > >>>
> > > >>> For MirrorMaker 1, we should have a KIP to deprecate its use in
> 2.6 if
> > > >> we want to remove it in 3.0.  I don't have a good sense of how
> > > practical it
> > > >> is to deprecate this now, so I will defer to others here.  But the
> KIP
> > > >> freeze for 2.6 is coming soon, so if we want to make the case, now
> is
> > > the
> > > >> time.
> > > >>>
> > > >>> best,
> > > >>> Colin
> > > >>>
> > > >>>
> > >  On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> > >  Hey folks,
> > > 
> > >  Sorry for stating that the bridge release would not break any
> > > >> compatibility
> > >  before, which is incorrect and confused many people.
> > > 
> > >  I think one way to think about the versioning is that:
> > > 
> > >  0) In a 2.x version moving ahead we would deprecate the
> ZK-dependent
> > > >> tools
> > >  such as --zookeeper flags from 

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Colin McCabe
Hi Michael,

It would be better to discuss the background behind KIP-500 in a separate 
thread, since this thread is about the Kafka 3.0 release.  As others have said, 
your questions are answered in the KIP.  For example, "what is the actual 
goal?" is addressed in the motivation section.

I agree that Kafka's usage of Apache ZooKeeper could be optimized.  But there 
are fundamental limitations to this approach compared to storing our metadata 
internally.  For example, having to contact a remote server to reload all your 
metadata on a controller failover simply doesn't scale past a certain point.

Apache Curator is a nice API, and if we were starting again today we would 
certainly consider using it.  But it doesn't allow us to do anything more 
efficiently than ZooKeeper could already do it.

Finally, Kafka's core competence is logs.  While our replication protocol is 
not Raft, it shares many similarities with that protocol.  So I think it's a 
bit unfair to say that it is "catastrophic hubris" to believe we can implement 
the protocol.

best,
Colin


On Sun, May 10, 2020, at 11:02, Michael K. Edwards wrote:
> Yes, I've read the KIP.  But all it really says to me is "we have never
> gotten around to using ZooKeeper properly."  To the extent that any of the
> distributed-state-maintenance problems discussed in "Metadata as an Event
> Log" can be solved — and some of them intrinsically can't, because CAP
> theorem — most of them are already implemented very effectively in Curator
> recipes.  (For instance, Curator's Tree Cache
> https://curator.apache.org/curator-recipes/tree-cache.html is a good fit to
> some of the state-maintenance needs.)
> 
> Kafka does have some usage patterns that don't map neatly onto existing
> Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> implements leader preference in the way that the existing Kafka partition
> leadership election procedure does.  But why not handle that by improving
> and extending Curator?  That way, other Curator users benefit, and we get
> additional highly experienced reviewers' eyes on the distributed
> algorithms, which are very very tricky to get right.
> 
> 
> On Sun, May 10, 2020 at 10:47 AM Ron Dagostino  wrote:
> 
> > Hi Michael.  This is discussed in the KIP.
> >
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> >
> > Ron
> >
> > > On May 10, 2020, at 1:35 PM, Michael K. Edwards 
> > wrote:
> > >
> > > What is the actual goal of removing the ZooKeeper dependency?  In my
> > > experience, if ZooKeeper is properly provisioned and deployed, it's
> > largely
> > > trouble-free.  (You do need to know how to use observers properly.)
> > There
> > > are some subtleties about timeouts and leadership changes, but they're
> > > pretty small stuff.  Why go to all the trouble of building a new
> > > distributed-consensus system that's going to have catastrophic bugs for
> > > years to come?  It seems like such an act of hubris to me, as well as a
> > > massive waste of engineering effort.  What is there to be gained?
> > >
> > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
> > wrote:
> > >>
> > >> Sure, we can compile a list for Kafka Streams. But the KIP would be for
> > >> 3.0, so I don't think it's urgent to do it now?
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > >>> Thanks, Guozhang-- sounds like a good plan.
> > >>>
> > >>> I think it would be good to have a list of deprecated streams APIs that
> > >> we want to remove in 3.0.  Maybe it's easiest to do that as its own KIP?
> > >>>
> > >>> For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6 if
> > >> we want to remove it in 3.0.  I don't have a good sense of how
> > practical it
> > >> is to deprecate this now, so I will defer to others here.  But the KIP
> > >> freeze for 2.6 is coming soon, so if we want to make the case, now is
> > the
> > >> time.
> > >>>
> > >>> best,
> > >>> Colin
> > >>>
> > >>>
> >  On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> >  Hey folks,
> > 
> >  Sorry for stating that the bridge release would not break any
> > >> compatibility
> >  before, which is incorrect and confused many people.
> > 
> >  I think one way to think about the versioning is that:
> > 
> >  0) In a 2.x version moving ahead we would deprecate the ZK-dependent
> > >> tools
> >  such as --zookeeper flags from various scripts (KIP-555)
> > 
> >  1) In 3.0 we would at least make one incompatible change for example
> > to
> >  remove the deprecated ZK flags.
> > 
> >  2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> >  including usages such as security credentials / broker registration /
> > >> etc
> >  which are via ZK today as well.
> > 
> >  Then for the bridge release(s), it 

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Andrew Otto
> Before we deprecate server side auto topic creation, we should have
client side auto topic creation for the producer:

Deprecating sounds fine, but before disabling it, it might be worthwhile to
wait long enough for non Java clients to catch up to this too.  :)

On Mon, May 11, 2020 at 4:45 PM Ismael Juma  wrote:

> Before we deprecate server side auto topic creation, we should have client
> side auto topic creation for the producer:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-487%3A+Client-side+Automatic+Topic+Creation+on+Producer
>
> Ismael
>
> On Mon, May 11, 2020 at 1:41 PM Colin McCabe  wrote:
>
> > On Mon, May 11, 2020, at 01:19, David Jacot wrote:
> > > Hi all,
> > >
> > > First, I agree with what has been discussed. Having 3.x as the bridge
> > > releases and entirely removing ZK in 4.0 makes total sense.
> > >
> > > Second, what would you think about removing the auto topics creation
> > > in 3.0? It is not recommended to use it anymore and that could simplify
> > > a bit our path towards removing ZK. We could deprecate it in 2.6
> already
> > > and remove it in 3.0. If you guys generally agree, I could draft a KIP
> > and
> > > kick off the discussion.
> > >
> >
> > I like the idea of removing automatic topic creation, but it seems like
> it
> > would be a big step, given that it's currently on by default.
> >
> > What about if we changed it to be off by default and deprecated in 3.0?
> > Then, people who hadn't yet changed their workflows could turn it on in
> > 3.0, and we could remove it entirely in 4.0.
> >
> > best,
> > Colin
> >
> > >
> > > Best,
> > > David
> > >
> > > On Sun, May 10, 2020 at 8:02 PM Michael K. Edwards <
> > m.k.edwa...@gmail.com>
> > > wrote:
> > >
> > > > Yes, I've read the KIP.  But all it really says to me is "we have
> never
> > > > gotten around to using ZooKeeper properly."  To the extent that any
> of
> > the
> > > > distributed-state-maintenance problems discussed in "Metadata as an
> > Event
> > > > Log" can be solved — and some of them intrinsically can't, because
> CAP
> > > > theorem — most of them are already implemented very effectively in
> > Curator
> > > > recipes.  (For instance, Curator's Tree Cache
> > > > https://curator.apache.org/curator-recipes/tree-cache.html is a good
> > fit
> > > > to
> > > > some of the state-maintenance needs.)
> > > >
> > > > Kafka does have some usage patterns that don't map neatly onto
> existing
> > > > Curator recipes.  For instance, neither LeaderSelector nor
> LeaderLatch
> > > > implements leader preference in the way that the existing Kafka
> > partition
> > > > leadership election procedure does.  But why not handle that by
> > improving
> > > > and extending Curator?  That way, other Curator users benefit, and we
> > get
> > > > additional highly experienced reviewers' eyes on the distributed
> > > > algorithms, which are very very tricky to get right.
> > > >
> > > >
> > > > On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
> > wrote:
> > > >
> > > > > Hi Michael.  This is discussed in the KIP.
> > > > >
> > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> > > > >
> > > > > Ron
> > > > >
> > > > > > On May 10, 2020, at 1:35 PM, Michael K. Edwards <
> > m.k.edwa...@gmail.com
> > > > >
> > > > > wrote:
> > > > > >
> > > > > > What is the actual goal of removing the ZooKeeper dependency?  In
> > my
> > > > > > experience, if ZooKeeper is properly provisioned and deployed,
> it's
> > > > > largely
> > > > > > trouble-free.  (You do need to know how to use observers
> properly.)
> > > > > There
> > > > > > are some subtleties about timeouts and leadership changes, but
> > they're
> > > > > > pretty small stuff.  Why go to all the trouble of building a new
> > > > > > distributed-consensus system that's going to have catastrophic
> > bugs for
> > > > > > years to come?  It seems like such an act of hubris to me, as
> well
> > as a
> > > > > > massive waste of engineering effort.  What is there to be gained?
> > > > > >
> > > > > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax <
> mj...@apache.org>
> > > > > wrote:
> > > > > >>
> > > > > >> Sure, we can compile a list for Kafka Streams. But the KIP would
> > be
> > > > for
> > > > > >> 3.0, so I don't think it's urgent to do it now?
> > > > > >>
> > > > > >>
> > > > > >> -Matthias
> > > > > >>
> > > > > >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > > > > >>> Thanks, Guozhang-- sounds like a good plan.
> > > > > >>>
> > > > > >>> I think it would be good to have a list of deprecated streams
> > APIs
> > > > that
> > > > > >> we want to remove in 3.0.  Maybe it's easiest to do that as its
> > own
> > > > KIP?
> > > > > >>>
> > > > > >>> For MirrorMaker 1, we should have a KIP to deprecate its use in
> > 2.6
> > > > if
> > > > > >> we want to remove it in 3.0.  I don't have a good sense of 

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Ismael Juma
Before we deprecate server side auto topic creation, we should have client
side auto topic creation for the producer:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-487%3A+Client-side+Automatic+Topic+Creation+on+Producer

Ismael

On Mon, May 11, 2020 at 1:41 PM Colin McCabe  wrote:

> On Mon, May 11, 2020, at 01:19, David Jacot wrote:
> > Hi all,
> >
> > First, I agree with what has been discussed. Having 3.x as the bridge
> > releases and entirely removing ZK in 4.0 makes total sense.
> >
> > Second, what would you think about removing the auto topics creation
> > in 3.0? It is not recommended to use it anymore and that could simplify
> > a bit our path towards removing ZK. We could deprecate it in 2.6 already
> > and remove it in 3.0. If you guys generally agree, I could draft a KIP
> and
> > kick off the discussion.
> >
>
> I like the idea of removing automatic topic creation, but it seems like it
> would be a big step, given that it's currently on by default.
>
> What about if we changed it to be off by default and deprecated in 3.0?
> Then, people who hadn't yet changed their workflows could turn it on in
> 3.0, and we could remove it entirely in 4.0.
>
> best,
> Colin
>
> >
> > Best,
> > David
> >
> > On Sun, May 10, 2020 at 8:02 PM Michael K. Edwards <
> m.k.edwa...@gmail.com>
> > wrote:
> >
> > > Yes, I've read the KIP.  But all it really says to me is "we have never
> > > gotten around to using ZooKeeper properly."  To the extent that any of
> the
> > > distributed-state-maintenance problems discussed in "Metadata as an
> Event
> > > Log" can be solved — and some of them intrinsically can't, because CAP
> > > theorem — most of them are already implemented very effectively in
> Curator
> > > recipes.  (For instance, Curator's Tree Cache
> > > https://curator.apache.org/curator-recipes/tree-cache.html is a good
> fit
> > > to
> > > some of the state-maintenance needs.)
> > >
> > > Kafka does have some usage patterns that don't map neatly onto existing
> > > Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> > > implements leader preference in the way that the existing Kafka
> partition
> > > leadership election procedure does.  But why not handle that by
> improving
> > > and extending Curator?  That way, other Curator users benefit, and we
> get
> > > additional highly experienced reviewers' eyes on the distributed
> > > algorithms, which are very very tricky to get right.
> > >
> > >
> > > On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
> wrote:
> > >
> > > > Hi Michael.  This is discussed in the KIP.
> > > >
> > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> > > >
> > > > Ron
> > > >
> > > > > On May 10, 2020, at 1:35 PM, Michael K. Edwards <
> m.k.edwa...@gmail.com
> > > >
> > > > wrote:
> > > > >
> > > > > What is the actual goal of removing the ZooKeeper dependency?  In
> my
> > > > > experience, if ZooKeeper is properly provisioned and deployed, it's
> > > > largely
> > > > > trouble-free.  (You do need to know how to use observers properly.)
> > > > There
> > > > > are some subtleties about timeouts and leadership changes, but
> they're
> > > > > pretty small stuff.  Why go to all the trouble of building a new
> > > > > distributed-consensus system that's going to have catastrophic
> bugs for
> > > > > years to come?  It seems like such an act of hubris to me, as well
> as a
> > > > > massive waste of engineering effort.  What is there to be gained?
> > > > >
> > > > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
> > > > wrote:
> > > > >>
> > > > >> Sure, we can compile a list for Kafka Streams. But the KIP would
> be
> > > for
> > > > >> 3.0, so I don't think it's urgent to do it now?
> > > > >>
> > > > >>
> > > > >> -Matthias
> > > > >>
> > > > >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > > > >>> Thanks, Guozhang-- sounds like a good plan.
> > > > >>>
> > > > >>> I think it would be good to have a list of deprecated streams
> APIs
> > > that
> > > > >> we want to remove in 3.0.  Maybe it's easiest to do that as its
> own
> > > KIP?
> > > > >>>
> > > > >>> For MirrorMaker 1, we should have a KIP to deprecate its use in
> 2.6
> > > if
> > > > >> we want to remove it in 3.0.  I don't have a good sense of how
> > > > practical it
> > > > >> is to deprecate this now, so I will defer to others here.  But
> the KIP
> > > > >> freeze for 2.6 is coming soon, so if we want to make the case,
> now is
> > > > the
> > > > >> time.
> > > > >>>
> > > > >>> best,
> > > > >>> Colin
> > > > >>>
> > > > >>>
> > > >  On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> > > >  Hey folks,
> > > > 
> > > >  Sorry for stating that the bridge release would not break any
> > > > >> compatibility
> > > >  before, which is incorrect and confused many people.
> > > > 
> > > >  I think one way to 

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Colin McCabe
On Mon, May 11, 2020, at 01:19, David Jacot wrote:
> Hi all,
> 
> First, I agree with what has been discussed. Having 3.x as the bridge
> releases and entirely removing ZK in 4.0 makes total sense.
> 
> Second, what would you think about removing the auto topics creation
> in 3.0? It is not recommended to use it anymore and that could simplify
> a bit our path towards removing ZK. We could deprecate it in 2.6 already
> and remove it in 3.0. If you guys generally agree, I could draft a KIP and
> kick off the discussion.
>

I like the idea of removing automatic topic creation, but it seems like it 
would be a big step, given that it's currently on by default.

What about if we changed it to be off by default and deprecated in 3.0?  Then, 
people who hadn't yet changed their workflows could turn it on in 3.0, and we 
could remove it entirely in 4.0.

best,
Colin

> 
> Best,
> David
> 
> On Sun, May 10, 2020 at 8:02 PM Michael K. Edwards 
> wrote:
> 
> > Yes, I've read the KIP.  But all it really says to me is "we have never
> > gotten around to using ZooKeeper properly."  To the extent that any of the
> > distributed-state-maintenance problems discussed in "Metadata as an Event
> > Log" can be solved — and some of them intrinsically can't, because CAP
> > theorem — most of them are already implemented very effectively in Curator
> > recipes.  (For instance, Curator's Tree Cache
> > https://curator.apache.org/curator-recipes/tree-cache.html is a good fit
> > to
> > some of the state-maintenance needs.)
> >
> > Kafka does have some usage patterns that don't map neatly onto existing
> > Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> > implements leader preference in the way that the existing Kafka partition
> > leadership election procedure does.  But why not handle that by improving
> > and extending Curator?  That way, other Curator users benefit, and we get
> > additional highly experienced reviewers' eyes on the distributed
> > algorithms, which are very very tricky to get right.
> >
> >
> > On Sun, May 10, 2020 at 10:47 AM Ron Dagostino  wrote:
> >
> > > Hi Michael.  This is discussed in the KIP.
> > >
> > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> > >
> > > Ron
> > >
> > > > On May 10, 2020, at 1:35 PM, Michael K. Edwards  > >
> > > wrote:
> > > >
> > > > What is the actual goal of removing the ZooKeeper dependency?  In my
> > > > experience, if ZooKeeper is properly provisioned and deployed, it's
> > > largely
> > > > trouble-free.  (You do need to know how to use observers properly.)
> > > There
> > > > are some subtleties about timeouts and leadership changes, but they're
> > > > pretty small stuff.  Why go to all the trouble of building a new
> > > > distributed-consensus system that's going to have catastrophic bugs for
> > > > years to come?  It seems like such an act of hubris to me, as well as a
> > > > massive waste of engineering effort.  What is there to be gained?
> > > >
> > > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
> > > wrote:
> > > >>
> > > >> Sure, we can compile a list for Kafka Streams. But the KIP would be
> > for
> > > >> 3.0, so I don't think it's urgent to do it now?
> > > >>
> > > >>
> > > >> -Matthias
> > > >>
> > > >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > > >>> Thanks, Guozhang-- sounds like a good plan.
> > > >>>
> > > >>> I think it would be good to have a list of deprecated streams APIs
> > that
> > > >> we want to remove in 3.0.  Maybe it's easiest to do that as its own
> > KIP?
> > > >>>
> > > >>> For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6
> > if
> > > >> we want to remove it in 3.0.  I don't have a good sense of how
> > > practical it
> > > >> is to deprecate this now, so I will defer to others here.  But the KIP
> > > >> freeze for 2.6 is coming soon, so if we want to make the case, now is
> > > the
> > > >> time.
> > > >>>
> > > >>> best,
> > > >>> Colin
> > > >>>
> > > >>>
> > >  On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> > >  Hey folks,
> > > 
> > >  Sorry for stating that the bridge release would not break any
> > > >> compatibility
> > >  before, which is incorrect and confused many people.
> > > 
> > >  I think one way to think about the versioning is that:
> > > 
> > >  0) In a 2.x version moving ahead we would deprecate the ZK-dependent
> > > >> tools
> > >  such as --zookeeper flags from various scripts (KIP-555)
> > > 
> > >  1) In 3.0 we would at least make one incompatible change for example
> > > to
> > >  remove the deprecated ZK flags.
> > > 
> > >  2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> > >  including usages such as security credentials / broker registration
> > /
> > > >> etc
> > >  which are via ZK today as well.
> > > 
> > > 

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Guozhang Wang
I'm +1 on deprecating the auto.topic.creation in 2.6 as well.


Guozhang

On Mon, May 11, 2020 at 1:20 AM David Jacot  wrote:

> Hi all,
>
> First, I agree with what has been discussed. Having 3.x as the bridge
> releases and entirely removing ZK in 4.0 makes total sense.
>
> Second, what would you think about removing the auto topics creation
> in 3.0? It is not recommended to use it anymore and that could simplify
> a bit our path towards removing ZK. We could deprecate it in 2.6 already
> and remove it in 3.0. If you guys generally agree, I could draft a KIP and
> kick off the discussion.
>
> Best,
> David
>
> On Sun, May 10, 2020 at 8:02 PM Michael K. Edwards 
> wrote:
>
> > Yes, I've read the KIP.  But all it really says to me is "we have never
> > gotten around to using ZooKeeper properly."  To the extent that any of
> the
> > distributed-state-maintenance problems discussed in "Metadata as an Event
> > Log" can be solved — and some of them intrinsically can't, because CAP
> > theorem — most of them are already implemented very effectively in
> Curator
> > recipes.  (For instance, Curator's Tree Cache
> > https://curator.apache.org/curator-recipes/tree-cache.html is a good fit
> > to
> > some of the state-maintenance needs.)
> >
> > Kafka does have some usage patterns that don't map neatly onto existing
> > Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> > implements leader preference in the way that the existing Kafka partition
> > leadership election procedure does.  But why not handle that by improving
> > and extending Curator?  That way, other Curator users benefit, and we get
> > additional highly experienced reviewers' eyes on the distributed
> > algorithms, which are very very tricky to get right.
> >
> >
> > On Sun, May 10, 2020 at 10:47 AM Ron Dagostino 
> wrote:
> >
> > > Hi Michael.  This is discussed in the KIP.
> > >
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> > >
> > > Ron
> > >
> > > > On May 10, 2020, at 1:35 PM, Michael K. Edwards <
> m.k.edwa...@gmail.com
> > >
> > > wrote:
> > > >
> > > > What is the actual goal of removing the ZooKeeper dependency?  In my
> > > > experience, if ZooKeeper is properly provisioned and deployed, it's
> > > largely
> > > > trouble-free.  (You do need to know how to use observers properly.)
> > > There
> > > > are some subtleties about timeouts and leadership changes, but
> they're
> > > > pretty small stuff.  Why go to all the trouble of building a new
> > > > distributed-consensus system that's going to have catastrophic bugs
> for
> > > > years to come?  It seems like such an act of hubris to me, as well
> as a
> > > > massive waste of engineering effort.  What is there to be gained?
> > > >
> > > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
> > > wrote:
> > > >>
> > > >> Sure, we can compile a list for Kafka Streams. But the KIP would be
> > for
> > > >> 3.0, so I don't think it's urgent to do it now?
> > > >>
> > > >>
> > > >> -Matthias
> > > >>
> > > >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > > >>> Thanks, Guozhang-- sounds like a good plan.
> > > >>>
> > > >>> I think it would be good to have a list of deprecated streams APIs
> > that
> > > >> we want to remove in 3.0.  Maybe it's easiest to do that as its own
> > KIP?
> > > >>>
> > > >>> For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6
> > if
> > > >> we want to remove it in 3.0.  I don't have a good sense of how
> > > practical it
> > > >> is to deprecate this now, so I will defer to others here.  But the
> KIP
> > > >> freeze for 2.6 is coming soon, so if we want to make the case, now
> is
> > > the
> > > >> time.
> > > >>>
> > > >>> best,
> > > >>> Colin
> > > >>>
> > > >>>
> > >  On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> > >  Hey folks,
> > > 
> > >  Sorry for stating that the bridge release would not break any
> > > >> compatibility
> > >  before, which is incorrect and confused many people.
> > > 
> > >  I think one way to think about the versioning is that:
> > > 
> > >  0) In a 2.x version moving ahead we would deprecate the
> ZK-dependent
> > > >> tools
> > >  such as --zookeeper flags from various scripts (KIP-555)
> > > 
> > >  1) In 3.0 we would at least make one incompatible change for
> example
> > > to
> > >  remove the deprecated ZK flags.
> > > 
> > >  2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> > >  including usages such as security credentials / broker
> registration
> > /
> > > >> etc
> > >  which are via ZK today as well.
> > > 
> > >  Then for the bridge release(s), it can be any or all of 3.x.
> > > 
> > > 
> > >  For 1), I'd love to add a few more incompatibility changes in 3.0
> > from
> > >  Kafka Streams: we evolve Streams public 

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread David Jacot
Hi all,

First, I agree with what has been discussed. Having 3.x as the bridge
releases and entirely removing ZK in 4.0 makes total sense.

Second, what would you think about removing the auto topics creation
in 3.0? It is not recommended to use it anymore and that could simplify
a bit our path towards removing ZK. We could deprecate it in 2.6 already
and remove it in 3.0. If you guys generally agree, I could draft a KIP and
kick off the discussion.

Best,
David

On Sun, May 10, 2020 at 8:02 PM Michael K. Edwards 
wrote:

> Yes, I've read the KIP.  But all it really says to me is "we have never
> gotten around to using ZooKeeper properly."  To the extent that any of the
> distributed-state-maintenance problems discussed in "Metadata as an Event
> Log" can be solved — and some of them intrinsically can't, because CAP
> theorem — most of them are already implemented very effectively in Curator
> recipes.  (For instance, Curator's Tree Cache
> https://curator.apache.org/curator-recipes/tree-cache.html is a good fit
> to
> some of the state-maintenance needs.)
>
> Kafka does have some usage patterns that don't map neatly onto existing
> Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
> implements leader preference in the way that the existing Kafka partition
> leadership election procedure does.  But why not handle that by improving
> and extending Curator?  That way, other Curator users benefit, and we get
> additional highly experienced reviewers' eyes on the distributed
> algorithms, which are very very tricky to get right.
>
>
> On Sun, May 10, 2020 at 10:47 AM Ron Dagostino  wrote:
>
> > Hi Michael.  This is discussed in the KIP.
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
> >
> > Ron
> >
> > > On May 10, 2020, at 1:35 PM, Michael K. Edwards  >
> > wrote:
> > >
> > > What is the actual goal of removing the ZooKeeper dependency?  In my
> > > experience, if ZooKeeper is properly provisioned and deployed, it's
> > largely
> > > trouble-free.  (You do need to know how to use observers properly.)
> > There
> > > are some subtleties about timeouts and leadership changes, but they're
> > > pretty small stuff.  Why go to all the trouble of building a new
> > > distributed-consensus system that's going to have catastrophic bugs for
> > > years to come?  It seems like such an act of hubris to me, as well as a
> > > massive waste of engineering effort.  What is there to be gained?
> > >
> > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
> > wrote:
> > >>
> > >> Sure, we can compile a list for Kafka Streams. But the KIP would be
> for
> > >> 3.0, so I don't think it's urgent to do it now?
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > >>> Thanks, Guozhang-- sounds like a good plan.
> > >>>
> > >>> I think it would be good to have a list of deprecated streams APIs
> that
> > >> we want to remove in 3.0.  Maybe it's easiest to do that as its own
> KIP?
> > >>>
> > >>> For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6
> if
> > >> we want to remove it in 3.0.  I don't have a good sense of how
> > practical it
> > >> is to deprecate this now, so I will defer to others here.  But the KIP
> > >> freeze for 2.6 is coming soon, so if we want to make the case, now is
> > the
> > >> time.
> > >>>
> > >>> best,
> > >>> Colin
> > >>>
> > >>>
> >  On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> >  Hey folks,
> > 
> >  Sorry for stating that the bridge release would not break any
> > >> compatibility
> >  before, which is incorrect and confused many people.
> > 
> >  I think one way to think about the versioning is that:
> > 
> >  0) In a 2.x version moving ahead we would deprecate the ZK-dependent
> > >> tools
> >  such as --zookeeper flags from various scripts (KIP-555)
> > 
> >  1) In 3.0 we would at least make one incompatible change for example
> > to
> >  remove the deprecated ZK flags.
> > 
> >  2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> >  including usages such as security credentials / broker registration
> /
> > >> etc
> >  which are via ZK today as well.
> > 
> >  Then for the bridge release(s), it can be any or all of 3.x.
> > 
> > 
> >  For 1), I'd love to add a few more incompatibility changes in 3.0
> from
> >  Kafka Streams: we evolve Streams public APIs by deprecating and then
> > >> remove
> >  in major releases, and since 2.0 we've accumulated quite a few
> > >> deprecated
> >  APIs, and I can compile a list of KIPs that contain those if people
> > are
> >  interested.
> > 
> > 
> >  Guozhang
> > 
> > 
> > > On Thu, May 7, 2020 at 3:53 PM Colin McCabe 
> > wrote:
> > >
> > > On Wed, May 6, 2020, at 21:33, Ryanne 

Re: [DISCUSS] Kafka 3.0

2020-05-10 Thread Michael K. Edwards
Yes, I've read the KIP.  But all it really says to me is "we have never
gotten around to using ZooKeeper properly."  To the extent that any of the
distributed-state-maintenance problems discussed in "Metadata as an Event
Log" can be solved — and some of them intrinsically can't, because CAP
theorem — most of them are already implemented very effectively in Curator
recipes.  (For instance, Curator's Tree Cache
https://curator.apache.org/curator-recipes/tree-cache.html is a good fit to
some of the state-maintenance needs.)

Kafka does have some usage patterns that don't map neatly onto existing
Curator recipes.  For instance, neither LeaderSelector nor LeaderLatch
implements leader preference in the way that the existing Kafka partition
leadership election procedure does.  But why not handle that by improving
and extending Curator?  That way, other Curator users benefit, and we get
additional highly experienced reviewers' eyes on the distributed
algorithms, which are very very tricky to get right.


On Sun, May 10, 2020 at 10:47 AM Ron Dagostino  wrote:

> Hi Michael.  This is discussed in the KIP.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum#KIP-500:ReplaceZooKeeperwithaSelf-ManagedMetadataQuorum-Motivation
>
> Ron
>
> > On May 10, 2020, at 1:35 PM, Michael K. Edwards 
> wrote:
> >
> > What is the actual goal of removing the ZooKeeper dependency?  In my
> > experience, if ZooKeeper is properly provisioned and deployed, it's
> largely
> > trouble-free.  (You do need to know how to use observers properly.)
> There
> > are some subtleties about timeouts and leadership changes, but they're
> > pretty small stuff.  Why go to all the trouble of building a new
> > distributed-consensus system that's going to have catastrophic bugs for
> > years to come?  It seems like such an act of hubris to me, as well as a
> > massive waste of engineering effort.  What is there to be gained?
> >
> >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax 
> wrote:
> >>
> >> Sure, we can compile a list for Kafka Streams. But the KIP would be for
> >> 3.0, so I don't think it's urgent to do it now?
> >>
> >>
> >> -Matthias
> >>
> >>> On 5/8/20 3:47 PM, Colin McCabe wrote:
> >>> Thanks, Guozhang-- sounds like a good plan.
> >>>
> >>> I think it would be good to have a list of deprecated streams APIs that
> >> we want to remove in 3.0.  Maybe it's easiest to do that as its own KIP?
> >>>
> >>> For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6 if
> >> we want to remove it in 3.0.  I don't have a good sense of how
> practical it
> >> is to deprecate this now, so I will defer to others here.  But the KIP
> >> freeze for 2.6 is coming soon, so if we want to make the case, now is
> the
> >> time.
> >>>
> >>> best,
> >>> Colin
> >>>
> >>>
>  On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
>  Hey folks,
> 
>  Sorry for stating that the bridge release would not break any
> >> compatibility
>  before, which is incorrect and confused many people.
> 
>  I think one way to think about the versioning is that:
> 
>  0) In a 2.x version moving ahead we would deprecate the ZK-dependent
> >> tools
>  such as --zookeeper flags from various scripts (KIP-555)
> 
>  1) In 3.0 we would at least make one incompatible change for example
> to
>  remove the deprecated ZK flags.
> 
>  2) In a future major version (e.g. 4.0) we would drop ZK entirely,
>  including usages such as security credentials / broker registration /
> >> etc
>  which are via ZK today as well.
> 
>  Then for the bridge release(s), it can be any or all of 3.x.
> 
> 
>  For 1), I'd love to add a few more incompatibility changes in 3.0 from
>  Kafka Streams: we evolve Streams public APIs by deprecating and then
> >> remove
>  in major releases, and since 2.0 we've accumulated quite a few
> >> deprecated
>  APIs, and I can compile a list of KIPs that contain those if people
> are
>  interested.
> 
> 
>  Guozhang
> 
> 
> > On Thu, May 7, 2020 at 3:53 PM Colin McCabe 
> wrote:
> >
> > On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
> >>> In fact, we know that the bridge release will involve at least one
> >>> incompatible change.  We will need to drop support for the
> >> --zookeeper
> >>> flags in the command-line tools.
> >>
> >> If the bridge release(s) and the subsequent post-ZK release are
> _both_
> >> breaking changes, I think we only have one option: the 3.x line are
> >> the
> >> bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
> >> Schofield.
> >>
> >> Specifically:
> >> - in order to _remove_ (not merely deprecate) the --zookeeper args,
> we
> > will
> >> need a major release.
> >> - in oder to drop support for ZK entirely (e.g. break a bunch of
> >> external
> >> tooling like 

Re: [DISCUSS] Kafka 3.0

2020-05-10 Thread Ron Dagostino
Hi Michael.  This is discussed in the KIP.

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

Ron

> On May 10, 2020, at 1:35 PM, Michael K. Edwards  wrote:
> 
> What is the actual goal of removing the ZooKeeper dependency?  In my
> experience, if ZooKeeper is properly provisioned and deployed, it's largely
> trouble-free.  (You do need to know how to use observers properly.)  There
> are some subtleties about timeouts and leadership changes, but they're
> pretty small stuff.  Why go to all the trouble of building a new
> distributed-consensus system that's going to have catastrophic bugs for
> years to come?  It seems like such an act of hubris to me, as well as a
> massive waste of engineering effort.  What is there to be gained?
> 
>> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax  wrote:
>> 
>> Sure, we can compile a list for Kafka Streams. But the KIP would be for
>> 3.0, so I don't think it's urgent to do it now?
>> 
>> 
>> -Matthias
>> 
>>> On 5/8/20 3:47 PM, Colin McCabe wrote:
>>> Thanks, Guozhang-- sounds like a good plan.
>>> 
>>> I think it would be good to have a list of deprecated streams APIs that
>> we want to remove in 3.0.  Maybe it's easiest to do that as its own KIP?
>>> 
>>> For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6 if
>> we want to remove it in 3.0.  I don't have a good sense of how practical it
>> is to deprecate this now, so I will defer to others here.  But the KIP
>> freeze for 2.6 is coming soon, so if we want to make the case, now is the
>> time.
>>> 
>>> best,
>>> Colin
>>> 
>>> 
 On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
 Hey folks,
 
 Sorry for stating that the bridge release would not break any
>> compatibility
 before, which is incorrect and confused many people.
 
 I think one way to think about the versioning is that:
 
 0) In a 2.x version moving ahead we would deprecate the ZK-dependent
>> tools
 such as --zookeeper flags from various scripts (KIP-555)
 
 1) In 3.0 we would at least make one incompatible change for example to
 remove the deprecated ZK flags.
 
 2) In a future major version (e.g. 4.0) we would drop ZK entirely,
 including usages such as security credentials / broker registration /
>> etc
 which are via ZK today as well.
 
 Then for the bridge release(s), it can be any or all of 3.x.
 
 
 For 1), I'd love to add a few more incompatibility changes in 3.0 from
 Kafka Streams: we evolve Streams public APIs by deprecating and then
>> remove
 in major releases, and since 2.0 we've accumulated quite a few
>> deprecated
 APIs, and I can compile a list of KIPs that contain those if people are
 interested.
 
 
 Guozhang
 
 
> On Thu, May 7, 2020 at 3:53 PM Colin McCabe  wrote:
> 
> On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
>>> In fact, we know that the bridge release will involve at least one
>>> incompatible change.  We will need to drop support for the
>> --zookeeper
>>> flags in the command-line tools.
>> 
>> If the bridge release(s) and the subsequent post-ZK release are _both_
>> breaking changes, I think we only have one option: the 3.x line are
>> the
>> bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
>> Schofield.
>> 
>> Specifically:
>> - in order to _remove_ (not merely deprecate) the --zookeeper args, we
> will
>> need a major release.
>> - in oder to drop support for ZK entirely (e.g. break a bunch of
>> external
>> tooling like Cruise Control), we will need a major release.
>> 
>> I count two major releases.
> 
> Hi Ryanne,
> 
> I agree that dropping ZK completely will need a new major release after
> 3.0.  I think that's OK and in keeping with how we've handled
>> deprecation
> and removal in the past.  It's important for users to have a smooth
>> upgrade
> path.
> 
> best,
> Colin
> 
>> 
>> Ryanne
>> 
>> -
>> 
>> On Wed, May 6, 2020 at 10:52 PM Colin McCabe 
>> wrote:
>> 
 On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
 Hey Colin, I think we should wait until after KIP-500's "bridge
 release" so there is a clean break from Zookeeper after 3.0. The
 bridge release by definition is an attempt to not break anything, so
 it theoretically doesn't warrant a major release.
>>> 
>>> Hi Ryanne,
>>> 
>>> I think it's important to clarify this a little bit.  The bridge
> release
>>> (really, releases, plural) allow you to upgrade from a cluster that
>> is
>>> using ZooKeeper to one that is not using ZooKeeper.  But, that
>> doesn't
>>> imply that the bridge release itself doesn't break anything.
>> Upgrading
>>> to 

Re: [DISCUSS] Kafka 3.0

2020-05-10 Thread Michael K. Edwards
What is the actual goal of removing the ZooKeeper dependency?  In my
experience, if ZooKeeper is properly provisioned and deployed, it's largely
trouble-free.  (You do need to know how to use observers properly.)  There
are some subtleties about timeouts and leadership changes, but they're
pretty small stuff.  Why go to all the trouble of building a new
distributed-consensus system that's going to have catastrophic bugs for
years to come?  It seems like such an act of hubris to me, as well as a
massive waste of engineering effort.  What is there to be gained?

On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax  wrote:

> Sure, we can compile a list for Kafka Streams. But the KIP would be for
> 3.0, so I don't think it's urgent to do it now?
>
>
> -Matthias
>
> On 5/8/20 3:47 PM, Colin McCabe wrote:
> > Thanks, Guozhang-- sounds like a good plan.
> >
> > I think it would be good to have a list of deprecated streams APIs that
> we want to remove in 3.0.  Maybe it's easiest to do that as its own KIP?
> >
> > For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6 if
> we want to remove it in 3.0.  I don't have a good sense of how practical it
> is to deprecate this now, so I will defer to others here.  But the KIP
> freeze for 2.6 is coming soon, so if we want to make the case, now is the
> time.
> >
> > best,
> > Colin
> >
> >
> > On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> >> Hey folks,
> >>
> >> Sorry for stating that the bridge release would not break any
> compatibility
> >> before, which is incorrect and confused many people.
> >>
> >> I think one way to think about the versioning is that:
> >>
> >> 0) In a 2.x version moving ahead we would deprecate the ZK-dependent
> tools
> >> such as --zookeeper flags from various scripts (KIP-555)
> >>
> >> 1) In 3.0 we would at least make one incompatible change for example to
> >> remove the deprecated ZK flags.
> >>
> >> 2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> >> including usages such as security credentials / broker registration /
> etc
> >> which are via ZK today as well.
> >>
> >> Then for the bridge release(s), it can be any or all of 3.x.
> >>
> >>
> >> For 1), I'd love to add a few more incompatibility changes in 3.0 from
> >> Kafka Streams: we evolve Streams public APIs by deprecating and then
> remove
> >> in major releases, and since 2.0 we've accumulated quite a few
> deprecated
> >> APIs, and I can compile a list of KIPs that contain those if people are
> >> interested.
> >>
> >>
> >> Guozhang
> >>
> >>
> >> On Thu, May 7, 2020 at 3:53 PM Colin McCabe  wrote:
> >>
> >>> On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
> > In fact, we know that the bridge release will involve at least one
> > incompatible change.  We will need to drop support for the
> --zookeeper
> > flags in the command-line tools.
> 
>  If the bridge release(s) and the subsequent post-ZK release are _both_
>  breaking changes, I think we only have one option: the 3.x line are
> the
>  bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
>  Schofield.
> 
>  Specifically:
>  - in order to _remove_ (not merely deprecate) the --zookeeper args, we
> >>> will
>  need a major release.
>  - in oder to drop support for ZK entirely (e.g. break a bunch of
> external
>  tooling like Cruise Control), we will need a major release.
> 
>  I count two major releases.
> >>>
> >>> Hi Ryanne,
> >>>
> >>> I agree that dropping ZK completely will need a new major release after
> >>> 3.0.  I think that's OK and in keeping with how we've handled
> deprecation
> >>> and removal in the past.  It's important for users to have a smooth
> upgrade
> >>> path.
> >>>
> >>> best,
> >>> Colin
> >>>
> 
>  Ryanne
> 
>  -
> 
>  On Wed, May 6, 2020 at 10:52 PM Colin McCabe 
> wrote:
> 
> > On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> >> Hey Colin, I think we should wait until after KIP-500's "bridge
> >> release" so there is a clean break from Zookeeper after 3.0. The
> >> bridge release by definition is an attempt to not break anything, so
> >> it theoretically doesn't warrant a major release.
> >
> > Hi Ryanne,
> >
> > I think it's important to clarify this a little bit.  The bridge
> >>> release
> > (really, releases, plural) allow you to upgrade from a cluster that
> is
> > using ZooKeeper to one that is not using ZooKeeper.  But, that
> doesn't
> > imply that the bridge release itself doesn't break anything.
> Upgrading
> > to the bridge release itself might involve some minor
> incompatibility.
> >
> > Kafka does occasionally have incompatible changes.  In those cases,
> we
> > bump the major version number.  One example is that when we went from
> > Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> > incompatible change.
> >
> > In fact, we know that the 

Re: [DISCUSS] Kafka 3.0

2020-05-08 Thread Matthias J. Sax
Sure, we can compile a list for Kafka Streams. But the KIP would be for
3.0, so I don't think it's urgent to do it now?


-Matthias

On 5/8/20 3:47 PM, Colin McCabe wrote:
> Thanks, Guozhang-- sounds like a good plan.
> 
> I think it would be good to have a list of deprecated streams APIs that we 
> want to remove in 3.0.  Maybe it's easiest to do that as its own KIP?
> 
> For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6 if we 
> want to remove it in 3.0.  I don't have a good sense of how practical it is 
> to deprecate this now, so I will defer to others here.  But the KIP freeze 
> for 2.6 is coming soon, so if we want to make the case, now is the time.
> 
> best,
> Colin
> 
> 
> On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
>> Hey folks,
>>
>> Sorry for stating that the bridge release would not break any compatibility
>> before, which is incorrect and confused many people.
>>
>> I think one way to think about the versioning is that:
>>
>> 0) In a 2.x version moving ahead we would deprecate the ZK-dependent tools
>> such as --zookeeper flags from various scripts (KIP-555)
>>
>> 1) In 3.0 we would at least make one incompatible change for example to
>> remove the deprecated ZK flags.
>>
>> 2) In a future major version (e.g. 4.0) we would drop ZK entirely,
>> including usages such as security credentials / broker registration / etc
>> which are via ZK today as well.
>>
>> Then for the bridge release(s), it can be any or all of 3.x.
>>
>>
>> For 1), I'd love to add a few more incompatibility changes in 3.0 from
>> Kafka Streams: we evolve Streams public APIs by deprecating and then remove
>> in major releases, and since 2.0 we've accumulated quite a few deprecated
>> APIs, and I can compile a list of KIPs that contain those if people are
>> interested.
>>
>>
>> Guozhang
>>
>>
>> On Thu, May 7, 2020 at 3:53 PM Colin McCabe  wrote:
>>
>>> On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
> In fact, we know that the bridge release will involve at least one
> incompatible change.  We will need to drop support for the --zookeeper
> flags in the command-line tools.

 If the bridge release(s) and the subsequent post-ZK release are _both_
 breaking changes, I think we only have one option: the 3.x line are the
 bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
 Schofield.

 Specifically:
 - in order to _remove_ (not merely deprecate) the --zookeeper args, we
>>> will
 need a major release.
 - in oder to drop support for ZK entirely (e.g. break a bunch of external
 tooling like Cruise Control), we will need a major release.

 I count two major releases.
>>>
>>> Hi Ryanne,
>>>
>>> I agree that dropping ZK completely will need a new major release after
>>> 3.0.  I think that's OK and in keeping with how we've handled deprecation
>>> and removal in the past.  It's important for users to have a smooth upgrade
>>> path.
>>>
>>> best,
>>> Colin
>>>

 Ryanne

 -

 On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:

> On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
>> Hey Colin, I think we should wait until after KIP-500's "bridge
>> release" so there is a clean break from Zookeeper after 3.0. The
>> bridge release by definition is an attempt to not break anything, so
>> it theoretically doesn't warrant a major release.
>
> Hi Ryanne,
>
> I think it's important to clarify this a little bit.  The bridge
>>> release
> (really, releases, plural) allow you to upgrade from a cluster that is
> using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
> imply that the bridge release itself doesn't break anything.  Upgrading
> to the bridge release itself might involve some minor incompatibility.
>
> Kafka does occasionally have incompatible changes.  In those cases, we
> bump the major version number.  One example is that when we went from
> Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> incompatible change.
>
> In fact, we know that the bridge release will involve at least one
> incompatible change.  We will need to drop support for the --zookeeper
> flags in the command-line tools.
>
> We've been preparing for this change for a long time.  People have
>>> spent
> a lot of effort designing new APIs that can be used instead of the old
> zookeeper-based code that some of the command-line tools used.  We have
> also deprecated the old ZK-based flags.  But at the end of the day, it
> is still an incompatible change.  So it's unfortunately not possible
>>> for
> the
> bridge release to be a 2.x release.
>
>> If that's not the case (i.e. if a single "bridge release" turns out
>>> to
>> be impractical), we should consider forking 3.0 while maintaining a
>> line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
>> 

Re: [DISCUSS] Kafka 3.0

2020-05-08 Thread Colin McCabe
Thanks, Guozhang-- sounds like a good plan.

I think it would be good to have a list of deprecated streams APIs that we want 
to remove in 3.0.  Maybe it's easiest to do that as its own KIP?

For MirrorMaker 1, we should have a KIP to deprecate its use in 2.6 if we want 
to remove it in 3.0.  I don't have a good sense of how practical it is to 
deprecate this now, so I will defer to others here.  But the KIP freeze for 2.6 
is coming soon, so if we want to make the case, now is the time.

best,
Colin


On Thu, May 7, 2020, at 16:28, Guozhang Wang wrote:
> Hey folks,
> 
> Sorry for stating that the bridge release would not break any compatibility
> before, which is incorrect and confused many people.
> 
> I think one way to think about the versioning is that:
> 
> 0) In a 2.x version moving ahead we would deprecate the ZK-dependent tools
> such as --zookeeper flags from various scripts (KIP-555)
> 
> 1) In 3.0 we would at least make one incompatible change for example to
> remove the deprecated ZK flags.
> 
> 2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> including usages such as security credentials / broker registration / etc
> which are via ZK today as well.
> 
> Then for the bridge release(s), it can be any or all of 3.x.
> 
> 
> For 1), I'd love to add a few more incompatibility changes in 3.0 from
> Kafka Streams: we evolve Streams public APIs by deprecating and then remove
> in major releases, and since 2.0 we've accumulated quite a few deprecated
> APIs, and I can compile a list of KIPs that contain those if people are
> interested.
> 
> 
> Guozhang
> 
> 
> On Thu, May 7, 2020 at 3:53 PM Colin McCabe  wrote:
> 
> > On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
> > > > In fact, we know that the bridge release will involve at least one
> > > > incompatible change.  We will need to drop support for the --zookeeper
> > > > flags in the command-line tools.
> > >
> > > If the bridge release(s) and the subsequent post-ZK release are _both_
> > > breaking changes, I think we only have one option: the 3.x line are the
> > > bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
> > > Schofield.
> > >
> > > Specifically:
> > > - in order to _remove_ (not merely deprecate) the --zookeeper args, we
> > will
> > > need a major release.
> > > - in oder to drop support for ZK entirely (e.g. break a bunch of external
> > > tooling like Cruise Control), we will need a major release.
> > >
> > > I count two major releases.
> >
> > Hi Ryanne,
> >
> > I agree that dropping ZK completely will need a new major release after
> > 3.0.  I think that's OK and in keeping with how we've handled deprecation
> > and removal in the past.  It's important for users to have a smooth upgrade
> > path.
> >
> > best,
> > Colin
> >
> > >
> > > Ryanne
> > >
> > > -
> > >
> > > On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:
> > >
> > > > On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> > > > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > > > release" so there is a clean break from Zookeeper after 3.0. The
> > > > > bridge release by definition is an attempt to not break anything, so
> > > > > it theoretically doesn't warrant a major release.
> > > >
> > > > Hi Ryanne,
> > > >
> > > > I think it's important to clarify this a little bit.  The bridge
> > release
> > > > (really, releases, plural) allow you to upgrade from a cluster that is
> > > > using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
> > > > imply that the bridge release itself doesn't break anything.  Upgrading
> > > > to the bridge release itself might involve some minor incompatibility.
> > > >
> > > > Kafka does occasionally have incompatible changes.  In those cases, we
> > > > bump the major version number.  One example is that when we went from
> > > > Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> > > > incompatible change.
> > > >
> > > > In fact, we know that the bridge release will involve at least one
> > > > incompatible change.  We will need to drop support for the --zookeeper
> > > > flags in the command-line tools.
> > > >
> > > > We've been preparing for this change for a long time.  People have
> > spent
> > > > a lot of effort designing new APIs that can be used instead of the old
> > > > zookeeper-based code that some of the command-line tools used.  We have
> > > > also deprecated the old ZK-based flags.  But at the end of the day, it
> > > > is still an incompatible change.  So it's unfortunately not possible
> > for
> > > > the
> > > > bridge release to be a 2.x release.
> > > >
> > > > > If that's not the case (i.e. if a single "bridge release" turns out
> > to
> > > > > be impractical), we should consider forking 3.0 while maintaining a
> > > > > line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> > > > > dramatically without breaking the 2.x line. In particular, anything
> > > > > related to removing Zookeeper could land 

Re: [DISCUSS] Kafka 3.0

2020-05-08 Thread Ryanne Dolan
Colin, Guozhang, Matthias, all makes sense to me, thanks.

Ryanne


On Fri, May 8, 2020, 1:48 PM Matthias J. Sax  wrote:

> I agree with what was discussed. Having a (breaking) 3.0 release to
> start the "bridge release" series and completing it with a 4.0 release
> sounds intuitive to me.
>
> For old MirrorMaker: In general I am also in favor of removing it, but
> there should be a reasonable deprecation period. Deprecating it in 2.6
> and removing it in 3.0 seems a little bit short to me? However, given
> that 4.0 would not come too much after 3.0 (not sure how many release we
> need to finalize ZK removal) it might be ok to have it a little longer?
>
> And yes, for Kafka Stream we would use the 3.0 release to remove a bunch
> of previously deprecated APIs that we accumulated since the 2.0 release.
>
>
> -Matthias
>
> On 5/7/20 4:28 PM, Guozhang Wang wrote:
> > Hey folks,
> >
> > Sorry for stating that the bridge release would not break any
> compatibility
> > before, which is incorrect and confused many people.
> >
> > I think one way to think about the versioning is that:
> >
> > 0) In a 2.x version moving ahead we would deprecate the ZK-dependent
> tools
> > such as --zookeeper flags from various scripts (KIP-555)
> >
> > 1) In 3.0 we would at least make one incompatible change for example to
> > remove the deprecated ZK flags.
> >
> > 2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> > including usages such as security credentials / broker registration / etc
> > which are via ZK today as well.
> >
> > Then for the bridge release(s), it can be any or all of 3.x.
> >
> >
> > For 1), I'd love to add a few more incompatibility changes in 3.0 from
> > Kafka Streams: we evolve Streams public APIs by deprecating and then
> remove
> > in major releases, and since 2.0 we've accumulated quite a few deprecated
> > APIs, and I can compile a list of KIPs that contain those if people are
> > interested.
> >
> >
> > Guozhang
> >
> >
> > On Thu, May 7, 2020 at 3:53 PM Colin McCabe  wrote:
> >
> >> On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
>  In fact, we know that the bridge release will involve at least one
>  incompatible change.  We will need to drop support for the --zookeeper
>  flags in the command-line tools.
> >>>
> >>> If the bridge release(s) and the subsequent post-ZK release are _both_
> >>> breaking changes, I think we only have one option: the 3.x line are the
> >>> bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
> >>> Schofield.
> >>>
> >>> Specifically:
> >>> - in order to _remove_ (not merely deprecate) the --zookeeper args, we
> >> will
> >>> need a major release.
> >>> - in oder to drop support for ZK entirely (e.g. break a bunch of
> external
> >>> tooling like Cruise Control), we will need a major release.
> >>>
> >>> I count two major releases.
> >>
> >> Hi Ryanne,
> >>
> >> I agree that dropping ZK completely will need a new major release after
> >> 3.0.  I think that's OK and in keeping with how we've handled
> deprecation
> >> and removal in the past.  It's important for users to have a smooth
> upgrade
> >> path.
> >>
> >> best,
> >> Colin
> >>
> >>>
> >>> Ryanne
> >>>
> >>> -
> >>>
> >>> On Wed, May 6, 2020 at 10:52 PM Colin McCabe 
> wrote:
> >>>
>  On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> > Hey Colin, I think we should wait until after KIP-500's "bridge
> > release" so there is a clean break from Zookeeper after 3.0. The
> > bridge release by definition is an attempt to not break anything, so
> > it theoretically doesn't warrant a major release.
> 
>  Hi Ryanne,
> 
>  I think it's important to clarify this a little bit.  The bridge
> >> release
>  (really, releases, plural) allow you to upgrade from a cluster that is
>  using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
>  imply that the bridge release itself doesn't break anything.
> Upgrading
>  to the bridge release itself might involve some minor incompatibility.
> 
>  Kafka does occasionally have incompatible changes.  In those cases, we
>  bump the major version number.  One example is that when we went from
>  Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
>  incompatible change.
> 
>  In fact, we know that the bridge release will involve at least one
>  incompatible change.  We will need to drop support for the --zookeeper
>  flags in the command-line tools.
> 
>  We've been preparing for this change for a long time.  People have
> >> spent
>  a lot of effort designing new APIs that can be used instead of the old
>  zookeeper-based code that some of the command-line tools used.  We
> have
>  also deprecated the old ZK-based flags.  But at the end of the day, it
>  is still an incompatible change.  So it's unfortunately not possible
> >> for
>  the
>  bridge release to be a 2.x release.
> 
> > If 

Re: [DISCUSS] Kafka 3.0

2020-05-08 Thread Matthias J. Sax
I agree with what was discussed. Having a (breaking) 3.0 release to
start the "bridge release" series and completing it with a 4.0 release
sounds intuitive to me.

For old MirrorMaker: In general I am also in favor of removing it, but
there should be a reasonable deprecation period. Deprecating it in 2.6
and removing it in 3.0 seems a little bit short to me? However, given
that 4.0 would not come too much after 3.0 (not sure how many release we
need to finalize ZK removal) it might be ok to have it a little longer?

And yes, for Kafka Stream we would use the 3.0 release to remove a bunch
of previously deprecated APIs that we accumulated since the 2.0 release.


-Matthias

On 5/7/20 4:28 PM, Guozhang Wang wrote:
> Hey folks,
> 
> Sorry for stating that the bridge release would not break any compatibility
> before, which is incorrect and confused many people.
> 
> I think one way to think about the versioning is that:
> 
> 0) In a 2.x version moving ahead we would deprecate the ZK-dependent tools
> such as --zookeeper flags from various scripts (KIP-555)
> 
> 1) In 3.0 we would at least make one incompatible change for example to
> remove the deprecated ZK flags.
> 
> 2) In a future major version (e.g. 4.0) we would drop ZK entirely,
> including usages such as security credentials / broker registration / etc
> which are via ZK today as well.
> 
> Then for the bridge release(s), it can be any or all of 3.x.
> 
> 
> For 1), I'd love to add a few more incompatibility changes in 3.0 from
> Kafka Streams: we evolve Streams public APIs by deprecating and then remove
> in major releases, and since 2.0 we've accumulated quite a few deprecated
> APIs, and I can compile a list of KIPs that contain those if people are
> interested.
> 
> 
> Guozhang
> 
> 
> On Thu, May 7, 2020 at 3:53 PM Colin McCabe  wrote:
> 
>> On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
 In fact, we know that the bridge release will involve at least one
 incompatible change.  We will need to drop support for the --zookeeper
 flags in the command-line tools.
>>>
>>> If the bridge release(s) and the subsequent post-ZK release are _both_
>>> breaking changes, I think we only have one option: the 3.x line are the
>>> bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
>>> Schofield.
>>>
>>> Specifically:
>>> - in order to _remove_ (not merely deprecate) the --zookeeper args, we
>> will
>>> need a major release.
>>> - in oder to drop support for ZK entirely (e.g. break a bunch of external
>>> tooling like Cruise Control), we will need a major release.
>>>
>>> I count two major releases.
>>
>> Hi Ryanne,
>>
>> I agree that dropping ZK completely will need a new major release after
>> 3.0.  I think that's OK and in keeping with how we've handled deprecation
>> and removal in the past.  It's important for users to have a smooth upgrade
>> path.
>>
>> best,
>> Colin
>>
>>>
>>> Ryanne
>>>
>>> -
>>>
>>> On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:
>>>
 On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> Hey Colin, I think we should wait until after KIP-500's "bridge
> release" so there is a clean break from Zookeeper after 3.0. The
> bridge release by definition is an attempt to not break anything, so
> it theoretically doesn't warrant a major release.

 Hi Ryanne,

 I think it's important to clarify this a little bit.  The bridge
>> release
 (really, releases, plural) allow you to upgrade from a cluster that is
 using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
 imply that the bridge release itself doesn't break anything.  Upgrading
 to the bridge release itself might involve some minor incompatibility.

 Kafka does occasionally have incompatible changes.  In those cases, we
 bump the major version number.  One example is that when we went from
 Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
 incompatible change.

 In fact, we know that the bridge release will involve at least one
 incompatible change.  We will need to drop support for the --zookeeper
 flags in the command-line tools.

 We've been preparing for this change for a long time.  People have
>> spent
 a lot of effort designing new APIs that can be used instead of the old
 zookeeper-based code that some of the command-line tools used.  We have
 also deprecated the old ZK-based flags.  But at the end of the day, it
 is still an incompatible change.  So it's unfortunately not possible
>> for
 the
 bridge release to be a 2.x release.

> If that's not the case (i.e. if a single "bridge release" turns out
>> to
> be impractical), we should consider forking 3.0 while maintaining a
> line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> dramatically without breaking the 2.x line. In particular, anything
> related to removing Zookeeper could land in pre-3.0 while every 

Re: [DISCUSS] Kafka 3.0

2020-05-07 Thread Guozhang Wang
Hey folks,

Sorry for stating that the bridge release would not break any compatibility
before, which is incorrect and confused many people.

I think one way to think about the versioning is that:

0) In a 2.x version moving ahead we would deprecate the ZK-dependent tools
such as --zookeeper flags from various scripts (KIP-555)

1) In 3.0 we would at least make one incompatible change for example to
remove the deprecated ZK flags.

2) In a future major version (e.g. 4.0) we would drop ZK entirely,
including usages such as security credentials / broker registration / etc
which are via ZK today as well.

Then for the bridge release(s), it can be any or all of 3.x.


For 1), I'd love to add a few more incompatibility changes in 3.0 from
Kafka Streams: we evolve Streams public APIs by deprecating and then remove
in major releases, and since 2.0 we've accumulated quite a few deprecated
APIs, and I can compile a list of KIPs that contain those if people are
interested.


Guozhang


On Thu, May 7, 2020 at 3:53 PM Colin McCabe  wrote:

> On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
> > > In fact, we know that the bridge release will involve at least one
> > > incompatible change.  We will need to drop support for the --zookeeper
> > > flags in the command-line tools.
> >
> > If the bridge release(s) and the subsequent post-ZK release are _both_
> > breaking changes, I think we only have one option: the 3.x line are the
> > bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
> > Schofield.
> >
> > Specifically:
> > - in order to _remove_ (not merely deprecate) the --zookeeper args, we
> will
> > need a major release.
> > - in oder to drop support for ZK entirely (e.g. break a bunch of external
> > tooling like Cruise Control), we will need a major release.
> >
> > I count two major releases.
>
> Hi Ryanne,
>
> I agree that dropping ZK completely will need a new major release after
> 3.0.  I think that's OK and in keeping with how we've handled deprecation
> and removal in the past.  It's important for users to have a smooth upgrade
> path.
>
> best,
> Colin
>
> >
> > Ryanne
> >
> > -
> >
> > On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:
> >
> > > On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> > > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > > release" so there is a clean break from Zookeeper after 3.0. The
> > > > bridge release by definition is an attempt to not break anything, so
> > > > it theoretically doesn't warrant a major release.
> > >
> > > Hi Ryanne,
> > >
> > > I think it's important to clarify this a little bit.  The bridge
> release
> > > (really, releases, plural) allow you to upgrade from a cluster that is
> > > using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
> > > imply that the bridge release itself doesn't break anything.  Upgrading
> > > to the bridge release itself might involve some minor incompatibility.
> > >
> > > Kafka does occasionally have incompatible changes.  In those cases, we
> > > bump the major version number.  One example is that when we went from
> > > Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> > > incompatible change.
> > >
> > > In fact, we know that the bridge release will involve at least one
> > > incompatible change.  We will need to drop support for the --zookeeper
> > > flags in the command-line tools.
> > >
> > > We've been preparing for this change for a long time.  People have
> spent
> > > a lot of effort designing new APIs that can be used instead of the old
> > > zookeeper-based code that some of the command-line tools used.  We have
> > > also deprecated the old ZK-based flags.  But at the end of the day, it
> > > is still an incompatible change.  So it's unfortunately not possible
> for
> > > the
> > > bridge release to be a 2.x release.
> > >
> > > > If that's not the case (i.e. if a single "bridge release" turns out
> to
> > > > be impractical), we should consider forking 3.0 while maintaining a
> > > > line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> > > > dramatically without breaking the 2.x line. In particular, anything
> > > > related to removing Zookeeper could land in pre-3.0 while every other
> > > > feature targets 2.6.
> > >
> > > Just to be super clear about this, what we want to do here is support
> > > operating in __either__ KIP-500 mode and legacy mode for a while.  So
> the
> > > same branch will have support for both the old way and the new way of
> > > managing metadata.
> > >
> > > This will allow us to get an "alpha" version of the KIP-500 mode out
> early
> > > for people to experiment with.  It also greatly reduces the number of
> Kafka
> > > releases we have to make, and the amount of backporting we have to do.
> > >
> > > >
> > > > If you are proposing 2.6 should be the "bridge release", I think this
> > > > is premature given Kafka's time-based release schedule. If the bridge
> > > > features happen to be 

Re: [DISCUSS] Kafka 3.0

2020-05-07 Thread Colin McCabe
On Wed, May 6, 2020, at 21:33, Ryanne Dolan wrote:
> > In fact, we know that the bridge release will involve at least one
> > incompatible change.  We will need to drop support for the --zookeeper
> > flags in the command-line tools.
> 
> If the bridge release(s) and the subsequent post-ZK release are _both_
> breaking changes, I think we only have one option: the 3.x line are the
> bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
> Schofield.
> 
> Specifically:
> - in order to _remove_ (not merely deprecate) the --zookeeper args, we will
> need a major release.
> - in oder to drop support for ZK entirely (e.g. break a bunch of external
> tooling like Cruise Control), we will need a major release.
> 
> I count two major releases.

Hi Ryanne,

I agree that dropping ZK completely will need a new major release after 3.0.  I 
think that's OK and in keeping with how we've handled deprecation and removal 
in the past.  It's important for users to have a smooth upgrade path.

best,
Colin

> 
> Ryanne
> 
> -
> 
> On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:
> 
> > On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > release" so there is a clean break from Zookeeper after 3.0. The
> > > bridge release by definition is an attempt to not break anything, so
> > > it theoretically doesn't warrant a major release.
> >
> > Hi Ryanne,
> >
> > I think it's important to clarify this a little bit.  The bridge release
> > (really, releases, plural) allow you to upgrade from a cluster that is
> > using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
> > imply that the bridge release itself doesn't break anything.  Upgrading
> > to the bridge release itself might involve some minor incompatibility.
> >
> > Kafka does occasionally have incompatible changes.  In those cases, we
> > bump the major version number.  One example is that when we went from
> > Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> > incompatible change.
> >
> > In fact, we know that the bridge release will involve at least one
> > incompatible change.  We will need to drop support for the --zookeeper
> > flags in the command-line tools.
> >
> > We've been preparing for this change for a long time.  People have spent
> > a lot of effort designing new APIs that can be used instead of the old
> > zookeeper-based code that some of the command-line tools used.  We have
> > also deprecated the old ZK-based flags.  But at the end of the day, it
> > is still an incompatible change.  So it's unfortunately not possible for
> > the
> > bridge release to be a 2.x release.
> >
> > > If that's not the case (i.e. if a single "bridge release" turns out to
> > > be impractical), we should consider forking 3.0 while maintaining a
> > > line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> > > dramatically without breaking the 2.x line. In particular, anything
> > > related to removing Zookeeper could land in pre-3.0 while every other
> > > feature targets 2.6.
> >
> > Just to be super clear about this, what we want to do here is support
> > operating in __either__ KIP-500 mode and legacy mode for a while.  So the
> > same branch will have support for both the old way and the new way of
> > managing metadata.
> >
> > This will allow us to get an "alpha" version of the KIP-500 mode out early
> > for people to experiment with.  It also greatly reduces the number of Kafka
> > releases we have to make, and the amount of backporting we have to do.
> >
> > >
> > > If you are proposing 2.6 should be the "bridge release", I think this
> > > is premature given Kafka's time-based release schedule. If the bridge
> > > features happen to be merged before 2.6's feature freeze, then sure --
> > > let's make that the bridge release in retrospect. And if we get all
> > > the post-Zookeeper features merged before 2.7, I'm onboard with naming
> > > it "3.0" instead.
> > >
> > > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> > > well. I'm happy to drive that additional breaking change. Maybe 2.6
> > > can be the "bridge" for MM2 as well.
> >
> > I don't have a strong opinion either way about this, but if we want to
> > remove the original MirrorMaker, we have to deprecate it first, right?  Are
> > we ready to do that?
> >
> > best,
> > Colin
> >
> > >
> > > Ryanne
> > >
> > > On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
> > >
> > > > Hi all,
> > > >
> > > > We've had a few proposals recently for incompatible changes.  One of
> > > > them is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > > > Tools.  The other is Boyang's KIP-590: Redirect ZK Mutation
> > > > Protocols to the Controller.  I think it's time to start thinking
> > > > about Kafka 3.0. Specifically, I think we should move to 3.0 after
> > > > the 2.6 release.
> > > >
> > > > From the perspective of KIP-500, in Kafka 3.x we'd like to make
> > > 

Re: [DISCUSS] Kafka 3.0

2020-05-07 Thread Colin McCabe
On Wed, May 6, 2020, at 21:40, Ryanne Dolan wrote:
> > This will allow us to get an "alpha" version of the KIP-500 mode out
> > early for people to experiment with
> 
> I think this is a non-sequitur. It's not a requirement that KIP-500 be
> merged to master and released in order for people to experiment with it.
>

Hi Ryanne,

I agree that it is not a requirement that KIP-500 be merged to master in order 
for people to experiment with it.  The main reason for merging to master is to 
avoid maintaining lots of branches and doing lots of backports.

>
> Nor does it sound great to call for a major release (3.0) in order to get
> an "alpha version ... out early".
> 

As I said earlier, the reason for the new major release is to make certain 
incompatible changes, not in order to get an alpha version of KIP-500 out.  For 
example, dropping the zookeeper flags is a step forward for security and 
encapsulation which also advances KIP-500.  Another example is that removing 
the kafka-preferred-replica-election.sh command removes a duplicate command 
that has been deprecated for a while.

best,
Colin

> 
> On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:
> 
> > On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > release" so there is a clean break from Zookeeper after 3.0. The
> > > bridge release by definition is an attempt to not break anything, so
> > > it theoretically doesn't warrant a major release.
> >
> > Hi Ryanne,
> >
> > I think it's important to clarify this a little bit.  The bridge release
> > (really, releases, plural) allow you to upgrade from a cluster that is
> > using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
> > imply that the bridge release itself doesn't break anything.  Upgrading
> > to the bridge release itself might involve some minor incompatibility.
> >
> > Kafka does occasionally have incompatible changes.  In those cases, we
> > bump the major version number.  One example is that when we went from
> > Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> > incompatible change.
> >
> > In fact, we know that the bridge release will involve at least one
> > incompatible change.  We will need to drop support for the --zookeeper
> > flags in the command-line tools.
> >
> > We've been preparing for this change for a long time.  People have spent
> > a lot of effort designing new APIs that can be used instead of the old
> > zookeeper-based code that some of the command-line tools used.  We have
> > also deprecated the old ZK-based flags.  But at the end of the day, it
> > is still an incompatible change.  So it's unfortunately not possible for
> > the
> > bridge release to be a 2.x release.
> >
> > > If that's not the case (i.e. if a single "bridge release" turns out to
> > > be impractical), we should consider forking 3.0 while maintaining a
> > > line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> > > dramatically without breaking the 2.x line. In particular, anything
> > > related to removing Zookeeper could land in pre-3.0 while every other
> > > feature targets 2.6.
> >
> > Just to be super clear about this, what we want to do here is support
> > operating in __either__ KIP-500 mode and legacy mode for a while.  So the
> > same branch will have support for both the old way and the new way of
> > managing metadata.
> >
> > This will allow us to get an "alpha" version of the KIP-500 mode out early
> > for people to experiment with.  It also greatly reduces the number of Kafka
> > releases we have to make, and the amount of backporting we have to do.
> >
> > >
> > > If you are proposing 2.6 should be the "bridge release", I think this
> > > is premature given Kafka's time-based release schedule. If the bridge
> > > features happen to be merged before 2.6's feature freeze, then sure --
> > > let's make that the bridge release in retrospect. And if we get all
> > > the post-Zookeeper features merged before 2.7, I'm onboard with naming
> > > it "3.0" instead.
> > >
> > > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> > > well. I'm happy to drive that additional breaking change. Maybe 2.6
> > > can be the "bridge" for MM2 as well.
> >
> > I don't have a strong opinion either way about this, but if we want to
> > remove the original MirrorMaker, we have to deprecate it first, right?  Are
> > we ready to do that?
> >
> > best,
> > Colin
> >
> > >
> > > Ryanne
> > >
> > > On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
> > >
> > > > Hi all,
> > > >
> > > > We've had a few proposals recently for incompatible changes.  One of
> > > > them is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > > > Tools.  The other is Boyang's KIP-590: Redirect ZK Mutation
> > > > Protocols to the Controller.  I think it's time to start thinking
> > > > about Kafka 3.0. Specifically, I think we should move to 3.0 after
> > > > the 2.6 release.
> > > >
> 

Re: [DISCUSS] Kafka 3.0

2020-05-06 Thread Ryanne Dolan
> This will allow us to get an "alpha" version of the KIP-500 mode out
early for people to experiment with

I think this is a non-sequitur. It's not a requirement that KIP-500 be
merged to master and released in order for people to experiment with it.
Nor does it sound great to call for a major release (3.0) in order to get
an "alpha version ... out early".

Ryanne

On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:

> On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> > Hey Colin, I think we should wait until after KIP-500's "bridge
> > release" so there is a clean break from Zookeeper after 3.0. The
> > bridge release by definition is an attempt to not break anything, so
> > it theoretically doesn't warrant a major release.
>
> Hi Ryanne,
>
> I think it's important to clarify this a little bit.  The bridge release
> (really, releases, plural) allow you to upgrade from a cluster that is
> using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
> imply that the bridge release itself doesn't break anything.  Upgrading
> to the bridge release itself might involve some minor incompatibility.
>
> Kafka does occasionally have incompatible changes.  In those cases, we
> bump the major version number.  One example is that when we went from
> Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> incompatible change.
>
> In fact, we know that the bridge release will involve at least one
> incompatible change.  We will need to drop support for the --zookeeper
> flags in the command-line tools.
>
> We've been preparing for this change for a long time.  People have spent
> a lot of effort designing new APIs that can be used instead of the old
> zookeeper-based code that some of the command-line tools used.  We have
> also deprecated the old ZK-based flags.  But at the end of the day, it
> is still an incompatible change.  So it's unfortunately not possible for
> the
> bridge release to be a 2.x release.
>
> > If that's not the case (i.e. if a single "bridge release" turns out to
> > be impractical), we should consider forking 3.0 while maintaining a
> > line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> > dramatically without breaking the 2.x line. In particular, anything
> > related to removing Zookeeper could land in pre-3.0 while every other
> > feature targets 2.6.
>
> Just to be super clear about this, what we want to do here is support
> operating in __either__ KIP-500 mode and legacy mode for a while.  So the
> same branch will have support for both the old way and the new way of
> managing metadata.
>
> This will allow us to get an "alpha" version of the KIP-500 mode out early
> for people to experiment with.  It also greatly reduces the number of Kafka
> releases we have to make, and the amount of backporting we have to do.
>
> >
> > If you are proposing 2.6 should be the "bridge release", I think this
> > is premature given Kafka's time-based release schedule. If the bridge
> > features happen to be merged before 2.6's feature freeze, then sure --
> > let's make that the bridge release in retrospect. And if we get all
> > the post-Zookeeper features merged before 2.7, I'm onboard with naming
> > it "3.0" instead.
> >
> > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> > well. I'm happy to drive that additional breaking change. Maybe 2.6
> > can be the "bridge" for MM2 as well.
>
> I don't have a strong opinion either way about this, but if we want to
> remove the original MirrorMaker, we have to deprecate it first, right?  Are
> we ready to do that?
>
> best,
> Colin
>
> >
> > Ryanne
> >
> > On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > We've had a few proposals recently for incompatible changes.  One of
> > > them is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > > Tools.  The other is Boyang's KIP-590: Redirect ZK Mutation
> > > Protocols to the Controller.  I think it's time to start thinking
> > > about Kafka 3.0. Specifically, I think we should move to 3.0 after
> > > the 2.6 release.
> > >
> > > From the perspective of KIP-500, in Kafka 3.x we'd like to make
> > > running in a ZooKeeper-less mode possible (but not yet the default.)
> > > This is the motivation behind KIP-590 and KIP-604, as well as some
> > > of the other KIPs we've done recently.  Since it will take some time
> > > to stabilize the new ZooKeeper-free Kafka code, we will hide it
> > > behind an option initially. (We'll have a KIP describing this all in
> > > detail soon.)
> > >
> > > What does everyone think about having Kafka 3.0 come up next after
> > > 2.6? Are there any other things we should change in the 2.6 -> 3.0
> > > transition?
> > >
> > > best, Colin
> > >
> >
>


Re: [DISCUSS] Kafka 3.0

2020-05-06 Thread Ryanne Dolan
> In fact, we know that the bridge release will involve at least one
> incompatible change.  We will need to drop support for the --zookeeper
> flags in the command-line tools.

If the bridge release(s) and the subsequent post-ZK release are _both_
breaking changes, I think we only have one option: the 3.x line are the
bridge release(s), and ZK is removed in 4.0, as suggested by Andrew
Schofield.

Specifically:
- in order to _remove_ (not merely deprecate) the --zookeeper args, we will
need a major release.
- in oder to drop support for ZK entirely (e.g. break a bunch of external
tooling like Cruise Control), we will need a major release.

I count two major releases.

Ryanne

-

On Wed, May 6, 2020 at 10:52 PM Colin McCabe  wrote:

> On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> > Hey Colin, I think we should wait until after KIP-500's "bridge
> > release" so there is a clean break from Zookeeper after 3.0. The
> > bridge release by definition is an attempt to not break anything, so
> > it theoretically doesn't warrant a major release.
>
> Hi Ryanne,
>
> I think it's important to clarify this a little bit.  The bridge release
> (really, releases, plural) allow you to upgrade from a cluster that is
> using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
> imply that the bridge release itself doesn't break anything.  Upgrading
> to the bridge release itself might involve some minor incompatibility.
>
> Kafka does occasionally have incompatible changes.  In those cases, we
> bump the major version number.  One example is that when we went from
> Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
> incompatible change.
>
> In fact, we know that the bridge release will involve at least one
> incompatible change.  We will need to drop support for the --zookeeper
> flags in the command-line tools.
>
> We've been preparing for this change for a long time.  People have spent
> a lot of effort designing new APIs that can be used instead of the old
> zookeeper-based code that some of the command-line tools used.  We have
> also deprecated the old ZK-based flags.  But at the end of the day, it
> is still an incompatible change.  So it's unfortunately not possible for
> the
> bridge release to be a 2.x release.
>
> > If that's not the case (i.e. if a single "bridge release" turns out to
> > be impractical), we should consider forking 3.0 while maintaining a
> > line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> > dramatically without breaking the 2.x line. In particular, anything
> > related to removing Zookeeper could land in pre-3.0 while every other
> > feature targets 2.6.
>
> Just to be super clear about this, what we want to do here is support
> operating in __either__ KIP-500 mode and legacy mode for a while.  So the
> same branch will have support for both the old way and the new way of
> managing metadata.
>
> This will allow us to get an "alpha" version of the KIP-500 mode out early
> for people to experiment with.  It also greatly reduces the number of Kafka
> releases we have to make, and the amount of backporting we have to do.
>
> >
> > If you are proposing 2.6 should be the "bridge release", I think this
> > is premature given Kafka's time-based release schedule. If the bridge
> > features happen to be merged before 2.6's feature freeze, then sure --
> > let's make that the bridge release in retrospect. And if we get all
> > the post-Zookeeper features merged before 2.7, I'm onboard with naming
> > it "3.0" instead.
> >
> > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> > well. I'm happy to drive that additional breaking change. Maybe 2.6
> > can be the "bridge" for MM2 as well.
>
> I don't have a strong opinion either way about this, but if we want to
> remove the original MirrorMaker, we have to deprecate it first, right?  Are
> we ready to do that?
>
> best,
> Colin
>
> >
> > Ryanne
> >
> > On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > We've had a few proposals recently for incompatible changes.  One of
> > > them is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > > Tools.  The other is Boyang's KIP-590: Redirect ZK Mutation
> > > Protocols to the Controller.  I think it's time to start thinking
> > > about Kafka 3.0. Specifically, I think we should move to 3.0 after
> > > the 2.6 release.
> > >
> > > From the perspective of KIP-500, in Kafka 3.x we'd like to make
> > > running in a ZooKeeper-less mode possible (but not yet the default.)
> > > This is the motivation behind KIP-590 and KIP-604, as well as some
> > > of the other KIPs we've done recently.  Since it will take some time
> > > to stabilize the new ZooKeeper-free Kafka code, we will hide it
> > > behind an option initially. (We'll have a KIP describing this all in
> > > detail soon.)
> > >
> > > What does everyone think about having Kafka 3.0 come up next after
> > > 2.6? Are there any other things we should change 

Re: [DISCUSS] Kafka 3.0

2020-05-06 Thread Colin McCabe
On Tue, May 5, 2020, at 12:36, Ryanne Dolan wrote:
> > In 3.0 it sounds like nothing is breaking and our big change won't be
> > complete... so, what's the motivation for the major release?
> 
> Exactly. Why would 3.1 be the breaking release? No one would expect
> everything to break going from 3.0 to 3.1
>

Hi Ryanne,

That's right.  Kafka uses Semantic Versioning.  See https://semver.org/.  The 
short summary is that we can't make incompatible changes in minor releases.

So the decision to move from 2.x to 3.0 isn't really about the number of new 
features or changes, but just about whether the new release is 100% compatible 
with the old.

Most users never encounter the incompatibilities, but it's important to set 
expectations that moving to a new 2.x release won't cause a compatibility 
break, but moving to a new major release might... IF you are using a deprecated 
feature.  Or sometimes if you are using an old JVM you will need to upgrade to 
upgrade major releases.

best,
Colin

> 
> On Tue, May 5, 2020 at 2:34 PM Gwen Shapira  wrote:
> 
> > It sounds like the decision to make the next release 3.0 is a bit arbitrary
> > then?
> >
> > With Exactly Once, we announced 1.0 as one release after the one where EOS
> > shipped, when we felt it was "ready" (little did we know... but that's
> > another story).
> > 2.0 was breaking due to us dropping Java 7.
> >
> > In 3.0 it sounds like nothing is breaking and our big change won't be
> > complete... so, what's the motivation for the major release?
> >
> > On Tue, May 5, 2020 at 12:12 PM Guozhang Wang  wrote:
> >
> > > I think there's a confusion regarding the "bridge release" proposed in
> > > KIP-500: should it be release "3.0" or be release "2.X" (i.e. the last
> > > minor release before 3.0).
> > >
> > > My understanding is that "3.0" would be the bridge release, i.e. it would
> > > not break any compatibility, but 3.1 potentially would, so an upgrade
> > from
> > > 2.5 to 3.1 would need to first upgrade to 3.0, and then to 3.1. For
> > > clients, all broker-client compatibility are still maintained 3.1+ so
> > that
> > > 2.x producer / consumer clients could still talk to 3.1+ brokers, only
> > > those old versioned scripts with on "--zookeeper" would not work with
> > 3.1+
> > > brokers anymore since there are no zookeepers.
> > >
> > >
> > > Guozhang
> > >
> > > On Mon, May 4, 2020 at 5:33 PM Gwen Shapira  wrote:
> > >
> > > > +1 for removing MM 1.0 when we cut a breaking release. It is sad to see
> > > > that we are still investing in it (I just saw a KIP toward improving
> > its
> > > > reset policy).
> > > >
> > > > My understanding was that KIP-590 is not breaking compatibility, I
> > think
> > > > Guozhang said that in response to my question on the discussion thread.
> > > >
> > > > Overall, since Kafka has time-based releases, we can make the call on
> > 3.0
> > > > vs 2.7 when we are at "KIP freeze date" and can see which features are
> > > > likely to make it.
> > > >
> > > >
> > > > On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan 
> > > wrote:
> > > >
> > > > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > release"
> > > > so
> > > > > there is a clean break from Zookeeper after 3.0. The bridge release
> > by
> > > > > definition is an attempt to not break anything, so it theoretically
> > > > doesn't
> > > > > warrant a major release. If that's not the case (i.e. if a single
> > > "bridge
> > > > > release" turns out to be impractical), we should consider forking 3.0
> > > > while
> > > > > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 3.x
> > > can
> > > > > evolve dramatically without breaking the 2.x line. In particular,
> > > > anything
> > > > > related to removing Zookeeper could land in pre-3.0 while every other
> > > > > feature targets 2.6.
> > > > >
> > > > > If you are proposing 2.6 should be the "bridge release", I think this
> > > is
> > > > > premature given Kafka's time-based release schedule. If the bridge
> > > > features
> > > > > happen to be merged before 2.6's feature freeze, then sure -- let's
> > > make
> > > > > that the bridge release in retrospect. And if we get all the
> > > > post-Zookeeper
> > > > > features merged before 2.7, I'm onboard with naming it "3.0" instead.
> > > > >
> > > > > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> > > well.
> > > > > I'm happy to drive that additional breaking change. Maybe 2.6 can be
> > > the
> > > > > "bridge" for MM2 as well.
> > > > >
> > > > > Ryanne
> > > > >
> > > > > On Mon, May 4, 2020, 5:05 PM Colin McCabe 
> > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > We've had a few proposals recently for incompatible changes.  One
> > of
> > > > them
> > > > > > is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > Tools.
> > > > The
> > > > > > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> > > > > > Controller.  I think it's time to start thinking about Kafka 3.0.
> 

Re: [DISCUSS] Kafka 3.0

2020-05-06 Thread Colin McCabe
On Mon, May 4, 2020, at 17:33, Gwen Shapira wrote:
> +1 for removing MM 1.0 when we cut a breaking release. It is sad to see
> that we are still investing in it (I just saw a KIP toward improving its
> reset policy).
> 
> My understanding was that KIP-590 is not breaking compatibility, I think
> Guozhang said that in response to my question on the discussion thread.

Hi Gwen,

The latest proposal for KIP-590 does break compatibility because it requires 
principals to be serializable.  So anyone implementing custom KafkaPrincipal 
subclasses would have to add support for such serialization.

> 
> Overall, since Kafka has time-based releases, we can make the call on 3.0
> vs 2.7 when we are at "KIP freeze date" and can see which features are
> likely to make it.
>

The release we're discussing won't happen until October or November, so I would 
put this more in the category of mid or long-term planning rather than short 
term planning.  It would be good to get some clarity on what we're going to do 
here.  If we can't drop support for the --zookeeper flags in November then I 
think the KIP-500 work will be delayed.  Remember that there are a lot of 
downstream users who won't migrate off of the --zookeeper flags until they're 
really gone-- things like k8s integrations, puppet, chef, and ansible 
integrations, and so on.

best,
Colin

> 
> 
> On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan  wrote:
> 
> > Hey Colin, I think we should wait until after KIP-500's "bridge release" so
> > there is a clean break from Zookeeper after 3.0. The bridge release by
> > definition is an attempt to not break anything, so it theoretically doesn't
> > warrant a major release. If that's not the case (i.e. if a single "bridge
> > release" turns out to be impractical), we should consider forking 3.0 while
> > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 3.x can
> > evolve dramatically without breaking the 2.x line. In particular, anything
> > related to removing Zookeeper could land in pre-3.0 while every other
> > feature targets 2.6.
> >
> > If you are proposing 2.6 should be the "bridge release", I think this is
> > premature given Kafka's time-based release schedule. If the bridge features
> > happen to be merged before 2.6's feature freeze, then sure -- let's make
> > that the bridge release in retrospect. And if we get all the post-Zookeeper
> > features merged before 2.7, I'm onboard with naming it "3.0" instead.
> >
> > That said, we should aim to remove legacy MirrorMaker before 3.0 as well.
> > I'm happy to drive that additional breaking change. Maybe 2.6 can be the
> > "bridge" for MM2 as well.
> >
> > Ryanne
> >
> > On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > We've had a few proposals recently for incompatible changes.  One of them
> > > is my KIP-604: Remove ZooKeeper Flags from the Administrative Tools.  The
> > > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> > > Controller.  I think it's time to start thinking about Kafka 3.0.
> > > Specifically, I think we should move to 3.0 after the 2.6 release.
> > >
> > > From the perspective of KIP-500, in Kafka 3.x we'd like to make running
> > in
> > > a ZooKeeper-less mode possible (but not yet the default.)  This is the
> > > motivation behind KIP-590 and KIP-604, as well as some of the other KIPs
> > > we've done recently.  Since it will take some time to stabilize the new
> > > ZooKeeper-free Kafka code, we will hide it behind an option initially.
> > > (We'll have a KIP describing this all in detail soon.)
> > >
> > > What does everyone think about having Kafka 3.0 come up next after 2.6?
> > > Are there any other things we should change in the 2.6 -> 3.0 transition?
> > >
> > > best,
> > > Colin
> > >
> >
> 
> 
> -- 
> Gwen Shapira
> Engineering Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>


Re: [DISCUSS] Kafka 3.0

2020-05-06 Thread Colin McCabe
On Mon, May 4, 2020, at 17:12, Ryanne Dolan wrote:
> Hey Colin, I think we should wait until after KIP-500's "bridge
> release" so there is a clean break from Zookeeper after 3.0. The
> bridge release by definition is an attempt to not break anything, so
> it theoretically doesn't warrant a major release.

Hi Ryanne,

I think it's important to clarify this a little bit.  The bridge release
(really, releases, plural) allow you to upgrade from a cluster that is
using ZooKeeper to one that is not using ZooKeeper.  But, that doesn't
imply that the bridge release itself doesn't break anything.  Upgrading
to the bridge release itself might involve some minor incompatibility.

Kafka does occasionally have incompatible changes.  In those cases, we
bump the major version number.  One example is that when we went from
Kafka 1.x to Kafka 2.0, we dropped support for JDK7.  This is an
incompatible change.

In fact, we know that the bridge release will involve at least one
incompatible change.  We will need to drop support for the --zookeeper
flags in the command-line tools.

We've been preparing for this change for a long time.  People have spent
a lot of effort designing new APIs that can be used instead of the old
zookeeper-based code that some of the command-line tools used.  We have
also deprecated the old ZK-based flags.  But at the end of the day, it
is still an incompatible change.  So it's unfortunately not possible for the
bridge release to be a 2.x release.

> If that's not the case (i.e. if a single "bridge release" turns out to
> be impractical), we should consider forking 3.0 while maintaining a
> line of Zookeeper-dependent Kafka in 2.x. That way 3.x can evolve
> dramatically without breaking the 2.x line. In particular, anything
> related to removing Zookeeper could land in pre-3.0 while every other
> feature targets 2.6.

Just to be super clear about this, what we want to do here is support operating 
in __either__ KIP-500 mode and legacy mode for a while.  So the same branch 
will have support for both the old way and the new way of managing metadata.

This will allow us to get an "alpha" version of the KIP-500 mode out early for 
people to experiment with.  It also greatly reduces the number of Kafka 
releases we have to make, and the amount of backporting we have to do.

>
> If you are proposing 2.6 should be the "bridge release", I think this
> is premature given Kafka's time-based release schedule. If the bridge
> features happen to be merged before 2.6's feature freeze, then sure --
> let's make that the bridge release in retrospect. And if we get all
> the post-Zookeeper features merged before 2.7, I'm onboard with naming
> it "3.0" instead.
>
> That said, we should aim to remove legacy MirrorMaker before 3.0 as
> well. I'm happy to drive that additional breaking change. Maybe 2.6
> can be the "bridge" for MM2 as well.

I don't have a strong opinion either way about this, but if we want to remove 
the original MirrorMaker, we have to deprecate it first, right?  Are we ready 
to do that?

best,
Colin

>
> Ryanne
>
> On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
>
> > Hi all,
> >
> > We've had a few proposals recently for incompatible changes.  One of
> > them is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > Tools.  The other is Boyang's KIP-590: Redirect ZK Mutation
> > Protocols to the Controller.  I think it's time to start thinking
> > about Kafka 3.0. Specifically, I think we should move to 3.0 after
> > the 2.6 release.
> >
> > From the perspective of KIP-500, in Kafka 3.x we'd like to make
> > running in a ZooKeeper-less mode possible (but not yet the default.)
> > This is the motivation behind KIP-590 and KIP-604, as well as some
> > of the other KIPs we've done recently.  Since it will take some time
> > to stabilize the new ZooKeeper-free Kafka code, we will hide it
> > behind an option initially. (We'll have a KIP describing this all in
> > detail soon.)
> >
> > What does everyone think about having Kafka 3.0 come up next after
> > 2.6? Are there any other things we should change in the 2.6 -> 3.0
> > transition?
> >
> > best, Colin
> >
>


RE: [DISCUSS] Kafka 3.0

2020-05-06 Thread Edoardo Comar
Ryanne Dolan  wrote on 05/05/2020 20:36:49:

> Exactly. Why would 3.1 be the breaking release? No one would expect
> everything to break going from 3.0 to 3.1

Agree completely

> 
> Ryanne
> 
> On Tue, May 5, 2020 at 2:34 PM Gwen Shapira  wrote:
> 
> > It sounds like the decision to make the next release 3.0 is a bit 
arbitrary
> > then?
> >
> > With Exactly Once, we announced 1.0 as one release after the one where 
EOS
> > shipped, when we felt it was "ready" (little did we know... but that's
> > another story).
> > 2.0 was breaking due to us dropping Java 7.
> >
> > In 3.0 it sounds like nothing is breaking and our big change won't be
> > complete... so, what's the motivation for the major release?
> >
> > On Tue, May 5, 2020 at 12:12 PM Guozhang Wang  
wrote:
> >
> > > I think there's a confusion regarding the "bridge release" proposed 
in
> > > KIP-500: should it be release "3.0" or be release "2.X" (i.e. the 
last
> > > minor release before 3.0).
> > >
> > > My understanding is that "3.0" would be the bridge release, i.e. it 
would
> > > not break any compatibility, but 3.1 potentially would, so an 
upgrade
> > from
> > > 2.5 to 3.1 would need to first upgrade to 3.0, and then to 3.1. For
> > > clients, all broker-client compatibility are still maintained 3.1+ 
so
> > that
> > > 2.x producer / consumer clients could still talk to 3.1+ brokers, 
only
> > > those old versioned scripts with on "--zookeeper" would not work 
with
> > 3.1+
> > > brokers anymore since there are no zookeepers.
> > >
> > >
> > > Guozhang
> > >
> > > On Mon, May 4, 2020 at 5:33 PM Gwen Shapira  
wrote:
> > >
> > > > +1 for removing MM 1.0 when we cut a breaking release. It is sad 
to see
> > > > that we are still investing in it (I just saw a KIP toward 
improving
> > its
> > > > reset policy).
> > > >
> > > > My understanding was that KIP-590 is not breaking compatibility, I
> > think
> > > > Guozhang said that in response to my question on the discussion 
thread.
> > > >
> > > > Overall, since Kafka has time-based releases, we can make the call 
on
> > 3.0
> > > > vs 2.7 when we are at "KIP freeze date" and can see which features 
are
> > > > likely to make it.
> > > >
> > > >
> > > > On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan 

> > > wrote:
> > > >
> > > > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > release"
> > > > so
> > > > > there is a clean break from Zookeeper after 3.0. The bridge 
release
> > by
> > > > > definition is an attempt to not break anything, so it 
theoretically
> > > > doesn't
> > > > > warrant a major release. If that's not the case (i.e. if a 
single
> > > "bridge
> > > > > release" turns out to be impractical), we should consider 
forking 3.0
> > > > while
> > > > > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 
3.x
> > > can
> > > > > evolve dramatically without breaking the 2.x line. In 
particular,
> > > > anything
> > > > > related to removing Zookeeper could land in pre-3.0 while every 
other
> > > > > feature targets 2.6.
> > > > >
> > > > > If you are proposing 2.6 should be the "bridge release", I think 
this
> > > is
> > > > > premature given Kafka's time-based release schedule. If the 
bridge
> > > > features
> > > > > happen to be merged before 2.6's feature freeze, then sure -- 
let's
> > > make
> > > > > that the bridge release in retrospect. And if we get all the
> > > > post-Zookeeper
> > > > > features merged before 2.7, I'm onboard with naming it "3.0" 
instead.
> > > > >
> > > > > That said, we should aim to remove legacy MirrorMaker before 3.0 
as
> > > well.
> > > > > I'm happy to drive that additional breaking change. Maybe 2.6 
can be
> > > the
> > > > > "bridge" for MM2 as well.
> > > > >
> > > > > Ryanne
> > > > >
> > > > > On Mon, May 4, 2020, 5:05 PM Colin McCabe 
> > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > We've had a few proposals recently for incompatible changes. 
One
> > of
> > > > them
> > > > > > is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > Tools.
> > > > The
> > > > > > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to 
the
> > > > > > Controller.  I think it's time to start thinking about Kafka 
3.0.
> > > > > > Specifically, I think we should move to 3.0 after the 2.6 
release.
> > > > > >
> > > > > > From the perspective of KIP-500, in Kafka 3.x we'd like to 
make
> > > running
> > > > > in
> > > > > > a ZooKeeper-less mode possible (but not yet the default.) This 
is
> > > the
> > > > > > motivation behind KIP-590 and KIP-604, as well as some of the 
other
> > > > KIPs
> > > > > > we've done recently.  Since it will take some time to 
stabilize the
> > > new
> > > > > > ZooKeeper-free Kafka code, we will hide it behind an option
> > > initially.
> > > > > > (We'll have a KIP describing this all in detail soon.)
> > > > > >
> > > > > > What does everyone think about having Kafka 3.0 come up next 
after
> > > 2.6?
> > > > > > Are there any other things we should 

Re: [DISCUSS] Kafka 3.0

2020-05-06 Thread Andrew Schofield
That's my view here. I think there are two ways this could work:

1) 2.x is the bridge release, and 3.0 is the one that completes the ZK removal.
2) 3.0 is the bridge release, and 4.0 is the one that completes the ZK removal.

On 06/05/2020, 05:15, "Jeff Widman"  wrote:

IMO a bridge release, or one that you have to upgrade to before upgrading
to the breaking release should be numbered as the last of the 2.x series...

In other words, it's acceptable to say "before upgrading to 3.0, first
upgrade to 2.9" but it's very unexpected to say "before upgrading to 3.1,
first upgrade to 3.0"... no one will be expecting that.

On Tue, May 5, 2020 at 12:37 PM Ryanne Dolan  wrote:

> > In 3.0 it sounds like nothing is breaking and our big change won't be
> > complete... so, what's the motivation for the major release?
>
> Exactly. Why would 3.1 be the breaking release? No one would expect
> everything to break going from 3.0 to 3.1
>
> Ryanne
>
> On Tue, May 5, 2020 at 2:34 PM Gwen Shapira  wrote:
>
> > It sounds like the decision to make the next release 3.0 is a bit
> arbitrary
> > then?
> >
> > With Exactly Once, we announced 1.0 as one release after the one where
> EOS
> > shipped, when we felt it was "ready" (little did we know... but that's
> > another story).
> > 2.0 was breaking due to us dropping Java 7.
> >
> > In 3.0 it sounds like nothing is breaking and our big change won't be
> > complete... so, what's the motivation for the major release?
> >
> > On Tue, May 5, 2020 at 12:12 PM Guozhang Wang 
> wrote:
> >
> > > I think there's a confusion regarding the "bridge release" proposed in
> > > KIP-500: should it be release "3.0" or be release "2.X" (i.e. the last
> > > minor release before 3.0).
> > >
> > > My understanding is that "3.0" would be the bridge release, i.e. it
> would
> > > not break any compatibility, but 3.1 potentially would, so an upgrade
> > from
> > > 2.5 to 3.1 would need to first upgrade to 3.0, and then to 3.1. For
> > > clients, all broker-client compatibility are still maintained 3.1+ so
> > that
> > > 2.x producer / consumer clients could still talk to 3.1+ brokers, only
> > > those old versioned scripts with on "--zookeeper" would not work with
> > 3.1+
> > > brokers anymore since there are no zookeepers.
> > >
> > >
> > > Guozhang
> > >
> > > On Mon, May 4, 2020 at 5:33 PM Gwen Shapira  wrote:
> > >
> > > > +1 for removing MM 1.0 when we cut a breaking release. It is sad to
> see
> > > > that we are still investing in it (I just saw a KIP toward improving
> > its
> > > > reset policy).
> > > >
> > > > My understanding was that KIP-590 is not breaking compatibility, I
> > think
> > > > Guozhang said that in response to my question on the discussion
> thread.
> > > >
> > > > Overall, since Kafka has time-based releases, we can make the call 
on
> > 3.0
> > > > vs 2.7 when we are at "KIP freeze date" and can see which features
> are
> > > > likely to make it.
> > > >
> > > >
> > > > On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan 
> > > wrote:
> > > >
> > > > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > release"
> > > > so
> > > > > there is a clean break from Zookeeper after 3.0. The bridge 
release
> > by
> > > > > definition is an attempt to not break anything, so it 
theoretically
> > > > doesn't
> > > > > warrant a major release. If that's not the case (i.e. if a single
> > > "bridge
> > > > > release" turns out to be impractical), we should consider forking
> 3.0
> > > > while
> > > > > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way
> 3.x
> > > can
> > > > > evolve dramatically without breaking the 2.x line. In particular,
> > > > anything
> > > > > related to removing Zookeeper could land in pre-3.0 while every
> other
> > > > > feature targets 2.6.
> > > > >
> > > > > If you are proposing 2.6 should be the "bridge release", I think
> this
> > > is
> > > > > premature given Kafka's time-based release schedule. If the bridge
> > > > features
> > > > > happen to be merged before 2.6's feature freeze, then sure -- 
let's
> > > make
> > > > > that the bridge release in retrospect. And if we get all the
> > > > post-Zookeeper
> > > > > features merged before 2.7, I'm onboard with naming it "3.0"
> instead.
> > > > >
> > > > > That said, we should aim to remove legacy MirrorMaker before 3.0 
as
> > > well.
> > > > > I'm happy to drive that additional breaking change. Maybe 2.6 can
> be
> > > the
> > > > > "bridge" for MM2 as well.
> > > > >
> > > > > Ryanne
> > > > >
> > > > > On 

Re: [DISCUSS] Kafka 3.0

2020-05-05 Thread Jeff Widman
IMO a bridge release, or one that you have to upgrade to before upgrading
to the breaking release should be numbered as the last of the 2.x series...

In other words, it's acceptable to say "before upgrading to 3.0, first
upgrade to 2.9" but it's very unexpected to say "before upgrading to 3.1,
first upgrade to 3.0"... no one will be expecting that.

On Tue, May 5, 2020 at 12:37 PM Ryanne Dolan  wrote:

> > In 3.0 it sounds like nothing is breaking and our big change won't be
> > complete... so, what's the motivation for the major release?
>
> Exactly. Why would 3.1 be the breaking release? No one would expect
> everything to break going from 3.0 to 3.1
>
> Ryanne
>
> On Tue, May 5, 2020 at 2:34 PM Gwen Shapira  wrote:
>
> > It sounds like the decision to make the next release 3.0 is a bit
> arbitrary
> > then?
> >
> > With Exactly Once, we announced 1.0 as one release after the one where
> EOS
> > shipped, when we felt it was "ready" (little did we know... but that's
> > another story).
> > 2.0 was breaking due to us dropping Java 7.
> >
> > In 3.0 it sounds like nothing is breaking and our big change won't be
> > complete... so, what's the motivation for the major release?
> >
> > On Tue, May 5, 2020 at 12:12 PM Guozhang Wang 
> wrote:
> >
> > > I think there's a confusion regarding the "bridge release" proposed in
> > > KIP-500: should it be release "3.0" or be release "2.X" (i.e. the last
> > > minor release before 3.0).
> > >
> > > My understanding is that "3.0" would be the bridge release, i.e. it
> would
> > > not break any compatibility, but 3.1 potentially would, so an upgrade
> > from
> > > 2.5 to 3.1 would need to first upgrade to 3.0, and then to 3.1. For
> > > clients, all broker-client compatibility are still maintained 3.1+ so
> > that
> > > 2.x producer / consumer clients could still talk to 3.1+ brokers, only
> > > those old versioned scripts with on "--zookeeper" would not work with
> > 3.1+
> > > brokers anymore since there are no zookeepers.
> > >
> > >
> > > Guozhang
> > >
> > > On Mon, May 4, 2020 at 5:33 PM Gwen Shapira  wrote:
> > >
> > > > +1 for removing MM 1.0 when we cut a breaking release. It is sad to
> see
> > > > that we are still investing in it (I just saw a KIP toward improving
> > its
> > > > reset policy).
> > > >
> > > > My understanding was that KIP-590 is not breaking compatibility, I
> > think
> > > > Guozhang said that in response to my question on the discussion
> thread.
> > > >
> > > > Overall, since Kafka has time-based releases, we can make the call on
> > 3.0
> > > > vs 2.7 when we are at "KIP freeze date" and can see which features
> are
> > > > likely to make it.
> > > >
> > > >
> > > > On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan 
> > > wrote:
> > > >
> > > > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > > release"
> > > > so
> > > > > there is a clean break from Zookeeper after 3.0. The bridge release
> > by
> > > > > definition is an attempt to not break anything, so it theoretically
> > > > doesn't
> > > > > warrant a major release. If that's not the case (i.e. if a single
> > > "bridge
> > > > > release" turns out to be impractical), we should consider forking
> 3.0
> > > > while
> > > > > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way
> 3.x
> > > can
> > > > > evolve dramatically without breaking the 2.x line. In particular,
> > > > anything
> > > > > related to removing Zookeeper could land in pre-3.0 while every
> other
> > > > > feature targets 2.6.
> > > > >
> > > > > If you are proposing 2.6 should be the "bridge release", I think
> this
> > > is
> > > > > premature given Kafka's time-based release schedule. If the bridge
> > > > features
> > > > > happen to be merged before 2.6's feature freeze, then sure -- let's
> > > make
> > > > > that the bridge release in retrospect. And if we get all the
> > > > post-Zookeeper
> > > > > features merged before 2.7, I'm onboard with naming it "3.0"
> instead.
> > > > >
> > > > > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> > > well.
> > > > > I'm happy to drive that additional breaking change. Maybe 2.6 can
> be
> > > the
> > > > > "bridge" for MM2 as well.
> > > > >
> > > > > Ryanne
> > > > >
> > > > > On Mon, May 4, 2020, 5:05 PM Colin McCabe 
> > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > We've had a few proposals recently for incompatible changes.  One
> > of
> > > > them
> > > > > > is my KIP-604: Remove ZooKeeper Flags from the Administrative
> > Tools.
> > > > The
> > > > > > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> > > > > > Controller.  I think it's time to start thinking about Kafka 3.0.
> > > > > > Specifically, I think we should move to 3.0 after the 2.6
> release.
> > > > > >
> > > > > > From the perspective of KIP-500, in Kafka 3.x we'd like to make
> > > running
> > > > > in
> > > > > > a ZooKeeper-less mode possible (but not yet the default.)  This
> is
> > > the
> > > > > > 

Re: [DISCUSS] Kafka 3.0

2020-05-05 Thread Ryanne Dolan
> In 3.0 it sounds like nothing is breaking and our big change won't be
> complete... so, what's the motivation for the major release?

Exactly. Why would 3.1 be the breaking release? No one would expect
everything to break going from 3.0 to 3.1

Ryanne

On Tue, May 5, 2020 at 2:34 PM Gwen Shapira  wrote:

> It sounds like the decision to make the next release 3.0 is a bit arbitrary
> then?
>
> With Exactly Once, we announced 1.0 as one release after the one where EOS
> shipped, when we felt it was "ready" (little did we know... but that's
> another story).
> 2.0 was breaking due to us dropping Java 7.
>
> In 3.0 it sounds like nothing is breaking and our big change won't be
> complete... so, what's the motivation for the major release?
>
> On Tue, May 5, 2020 at 12:12 PM Guozhang Wang  wrote:
>
> > I think there's a confusion regarding the "bridge release" proposed in
> > KIP-500: should it be release "3.0" or be release "2.X" (i.e. the last
> > minor release before 3.0).
> >
> > My understanding is that "3.0" would be the bridge release, i.e. it would
> > not break any compatibility, but 3.1 potentially would, so an upgrade
> from
> > 2.5 to 3.1 would need to first upgrade to 3.0, and then to 3.1. For
> > clients, all broker-client compatibility are still maintained 3.1+ so
> that
> > 2.x producer / consumer clients could still talk to 3.1+ brokers, only
> > those old versioned scripts with on "--zookeeper" would not work with
> 3.1+
> > brokers anymore since there are no zookeepers.
> >
> >
> > Guozhang
> >
> > On Mon, May 4, 2020 at 5:33 PM Gwen Shapira  wrote:
> >
> > > +1 for removing MM 1.0 when we cut a breaking release. It is sad to see
> > > that we are still investing in it (I just saw a KIP toward improving
> its
> > > reset policy).
> > >
> > > My understanding was that KIP-590 is not breaking compatibility, I
> think
> > > Guozhang said that in response to my question on the discussion thread.
> > >
> > > Overall, since Kafka has time-based releases, we can make the call on
> 3.0
> > > vs 2.7 when we are at "KIP freeze date" and can see which features are
> > > likely to make it.
> > >
> > >
> > > On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan 
> > wrote:
> > >
> > > > Hey Colin, I think we should wait until after KIP-500's "bridge
> > release"
> > > so
> > > > there is a clean break from Zookeeper after 3.0. The bridge release
> by
> > > > definition is an attempt to not break anything, so it theoretically
> > > doesn't
> > > > warrant a major release. If that's not the case (i.e. if a single
> > "bridge
> > > > release" turns out to be impractical), we should consider forking 3.0
> > > while
> > > > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 3.x
> > can
> > > > evolve dramatically without breaking the 2.x line. In particular,
> > > anything
> > > > related to removing Zookeeper could land in pre-3.0 while every other
> > > > feature targets 2.6.
> > > >
> > > > If you are proposing 2.6 should be the "bridge release", I think this
> > is
> > > > premature given Kafka's time-based release schedule. If the bridge
> > > features
> > > > happen to be merged before 2.6's feature freeze, then sure -- let's
> > make
> > > > that the bridge release in retrospect. And if we get all the
> > > post-Zookeeper
> > > > features merged before 2.7, I'm onboard with naming it "3.0" instead.
> > > >
> > > > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> > well.
> > > > I'm happy to drive that additional breaking change. Maybe 2.6 can be
> > the
> > > > "bridge" for MM2 as well.
> > > >
> > > > Ryanne
> > > >
> > > > On Mon, May 4, 2020, 5:05 PM Colin McCabe 
> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > We've had a few proposals recently for incompatible changes.  One
> of
> > > them
> > > > > is my KIP-604: Remove ZooKeeper Flags from the Administrative
> Tools.
> > > The
> > > > > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> > > > > Controller.  I think it's time to start thinking about Kafka 3.0.
> > > > > Specifically, I think we should move to 3.0 after the 2.6 release.
> > > > >
> > > > > From the perspective of KIP-500, in Kafka 3.x we'd like to make
> > running
> > > > in
> > > > > a ZooKeeper-less mode possible (but not yet the default.)  This is
> > the
> > > > > motivation behind KIP-590 and KIP-604, as well as some of the other
> > > KIPs
> > > > > we've done recently.  Since it will take some time to stabilize the
> > new
> > > > > ZooKeeper-free Kafka code, we will hide it behind an option
> > initially.
> > > > > (We'll have a KIP describing this all in detail soon.)
> > > > >
> > > > > What does everyone think about having Kafka 3.0 come up next after
> > 2.6?
> > > > > Are there any other things we should change in the 2.6 -> 3.0
> > > transition?
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > >
> > >
> > >
> > > --
> > > Gwen Shapira
> > > Engineering Manager | Confluent
> > > 650.450.2760 | @gwenshap
> > > 

Re: [DISCUSS] Kafka 3.0

2020-05-05 Thread Gwen Shapira
It sounds like the decision to make the next release 3.0 is a bit arbitrary
then?

With Exactly Once, we announced 1.0 as one release after the one where EOS
shipped, when we felt it was "ready" (little did we know... but that's
another story).
2.0 was breaking due to us dropping Java 7.

In 3.0 it sounds like nothing is breaking and our big change won't be
complete... so, what's the motivation for the major release?

On Tue, May 5, 2020 at 12:12 PM Guozhang Wang  wrote:

> I think there's a confusion regarding the "bridge release" proposed in
> KIP-500: should it be release "3.0" or be release "2.X" (i.e. the last
> minor release before 3.0).
>
> My understanding is that "3.0" would be the bridge release, i.e. it would
> not break any compatibility, but 3.1 potentially would, so an upgrade from
> 2.5 to 3.1 would need to first upgrade to 3.0, and then to 3.1. For
> clients, all broker-client compatibility are still maintained 3.1+ so that
> 2.x producer / consumer clients could still talk to 3.1+ brokers, only
> those old versioned scripts with on "--zookeeper" would not work with 3.1+
> brokers anymore since there are no zookeepers.
>
>
> Guozhang
>
> On Mon, May 4, 2020 at 5:33 PM Gwen Shapira  wrote:
>
> > +1 for removing MM 1.0 when we cut a breaking release. It is sad to see
> > that we are still investing in it (I just saw a KIP toward improving its
> > reset policy).
> >
> > My understanding was that KIP-590 is not breaking compatibility, I think
> > Guozhang said that in response to my question on the discussion thread.
> >
> > Overall, since Kafka has time-based releases, we can make the call on 3.0
> > vs 2.7 when we are at "KIP freeze date" and can see which features are
> > likely to make it.
> >
> >
> > On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan 
> wrote:
> >
> > > Hey Colin, I think we should wait until after KIP-500's "bridge
> release"
> > so
> > > there is a clean break from Zookeeper after 3.0. The bridge release by
> > > definition is an attempt to not break anything, so it theoretically
> > doesn't
> > > warrant a major release. If that's not the case (i.e. if a single
> "bridge
> > > release" turns out to be impractical), we should consider forking 3.0
> > while
> > > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 3.x
> can
> > > evolve dramatically without breaking the 2.x line. In particular,
> > anything
> > > related to removing Zookeeper could land in pre-3.0 while every other
> > > feature targets 2.6.
> > >
> > > If you are proposing 2.6 should be the "bridge release", I think this
> is
> > > premature given Kafka's time-based release schedule. If the bridge
> > features
> > > happen to be merged before 2.6's feature freeze, then sure -- let's
> make
> > > that the bridge release in retrospect. And if we get all the
> > post-Zookeeper
> > > features merged before 2.7, I'm onboard with naming it "3.0" instead.
> > >
> > > That said, we should aim to remove legacy MirrorMaker before 3.0 as
> well.
> > > I'm happy to drive that additional breaking change. Maybe 2.6 can be
> the
> > > "bridge" for MM2 as well.
> > >
> > > Ryanne
> > >
> > > On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
> > >
> > > > Hi all,
> > > >
> > > > We've had a few proposals recently for incompatible changes.  One of
> > them
> > > > is my KIP-604: Remove ZooKeeper Flags from the Administrative Tools.
> > The
> > > > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> > > > Controller.  I think it's time to start thinking about Kafka 3.0.
> > > > Specifically, I think we should move to 3.0 after the 2.6 release.
> > > >
> > > > From the perspective of KIP-500, in Kafka 3.x we'd like to make
> running
> > > in
> > > > a ZooKeeper-less mode possible (but not yet the default.)  This is
> the
> > > > motivation behind KIP-590 and KIP-604, as well as some of the other
> > KIPs
> > > > we've done recently.  Since it will take some time to stabilize the
> new
> > > > ZooKeeper-free Kafka code, we will hide it behind an option
> initially.
> > > > (We'll have a KIP describing this all in detail soon.)
> > > >
> > > > What does everyone think about having Kafka 3.0 come up next after
> 2.6?
> > > > Are there any other things we should change in the 2.6 -> 3.0
> > transition?
> > > >
> > > > best,
> > > > Colin
> > > >
> > >
> >
> >
> > --
> > Gwen Shapira
> > Engineering Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter | blog
> >
>
>
> --
> -- Guozhang
>


-- 
Gwen Shapira
Engineering Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog


Re: [DISCUSS] Kafka 3.0

2020-05-05 Thread Guozhang Wang
I think there's a confusion regarding the "bridge release" proposed in
KIP-500: should it be release "3.0" or be release "2.X" (i.e. the last
minor release before 3.0).

My understanding is that "3.0" would be the bridge release, i.e. it would
not break any compatibility, but 3.1 potentially would, so an upgrade from
2.5 to 3.1 would need to first upgrade to 3.0, and then to 3.1. For
clients, all broker-client compatibility are still maintained 3.1+ so that
2.x producer / consumer clients could still talk to 3.1+ brokers, only
those old versioned scripts with on "--zookeeper" would not work with 3.1+
brokers anymore since there are no zookeepers.


Guozhang

On Mon, May 4, 2020 at 5:33 PM Gwen Shapira  wrote:

> +1 for removing MM 1.0 when we cut a breaking release. It is sad to see
> that we are still investing in it (I just saw a KIP toward improving its
> reset policy).
>
> My understanding was that KIP-590 is not breaking compatibility, I think
> Guozhang said that in response to my question on the discussion thread.
>
> Overall, since Kafka has time-based releases, we can make the call on 3.0
> vs 2.7 when we are at "KIP freeze date" and can see which features are
> likely to make it.
>
>
> On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan  wrote:
>
> > Hey Colin, I think we should wait until after KIP-500's "bridge release"
> so
> > there is a clean break from Zookeeper after 3.0. The bridge release by
> > definition is an attempt to not break anything, so it theoretically
> doesn't
> > warrant a major release. If that's not the case (i.e. if a single "bridge
> > release" turns out to be impractical), we should consider forking 3.0
> while
> > maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 3.x can
> > evolve dramatically without breaking the 2.x line. In particular,
> anything
> > related to removing Zookeeper could land in pre-3.0 while every other
> > feature targets 2.6.
> >
> > If you are proposing 2.6 should be the "bridge release", I think this is
> > premature given Kafka's time-based release schedule. If the bridge
> features
> > happen to be merged before 2.6's feature freeze, then sure -- let's make
> > that the bridge release in retrospect. And if we get all the
> post-Zookeeper
> > features merged before 2.7, I'm onboard with naming it "3.0" instead.
> >
> > That said, we should aim to remove legacy MirrorMaker before 3.0 as well.
> > I'm happy to drive that additional breaking change. Maybe 2.6 can be the
> > "bridge" for MM2 as well.
> >
> > Ryanne
> >
> > On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > We've had a few proposals recently for incompatible changes.  One of
> them
> > > is my KIP-604: Remove ZooKeeper Flags from the Administrative Tools.
> The
> > > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> > > Controller.  I think it's time to start thinking about Kafka 3.0.
> > > Specifically, I think we should move to 3.0 after the 2.6 release.
> > >
> > > From the perspective of KIP-500, in Kafka 3.x we'd like to make running
> > in
> > > a ZooKeeper-less mode possible (but not yet the default.)  This is the
> > > motivation behind KIP-590 and KIP-604, as well as some of the other
> KIPs
> > > we've done recently.  Since it will take some time to stabilize the new
> > > ZooKeeper-free Kafka code, we will hide it behind an option initially.
> > > (We'll have a KIP describing this all in detail soon.)
> > >
> > > What does everyone think about having Kafka 3.0 come up next after 2.6?
> > > Are there any other things we should change in the 2.6 -> 3.0
> transition?
> > >
> > > best,
> > > Colin
> > >
> >
>
>
> --
> Gwen Shapira
> Engineering Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>


-- 
-- Guozhang


Re: [DISCUSS] Kafka 3.0

2020-05-04 Thread Gwen Shapira
+1 for removing MM 1.0 when we cut a breaking release. It is sad to see
that we are still investing in it (I just saw a KIP toward improving its
reset policy).

My understanding was that KIP-590 is not breaking compatibility, I think
Guozhang said that in response to my question on the discussion thread.

Overall, since Kafka has time-based releases, we can make the call on 3.0
vs 2.7 when we are at "KIP freeze date" and can see which features are
likely to make it.


On Mon, May 4, 2020 at 5:13 PM Ryanne Dolan  wrote:

> Hey Colin, I think we should wait until after KIP-500's "bridge release" so
> there is a clean break from Zookeeper after 3.0. The bridge release by
> definition is an attempt to not break anything, so it theoretically doesn't
> warrant a major release. If that's not the case (i.e. if a single "bridge
> release" turns out to be impractical), we should consider forking 3.0 while
> maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 3.x can
> evolve dramatically without breaking the 2.x line. In particular, anything
> related to removing Zookeeper could land in pre-3.0 while every other
> feature targets 2.6.
>
> If you are proposing 2.6 should be the "bridge release", I think this is
> premature given Kafka's time-based release schedule. If the bridge features
> happen to be merged before 2.6's feature freeze, then sure -- let's make
> that the bridge release in retrospect. And if we get all the post-Zookeeper
> features merged before 2.7, I'm onboard with naming it "3.0" instead.
>
> That said, we should aim to remove legacy MirrorMaker before 3.0 as well.
> I'm happy to drive that additional breaking change. Maybe 2.6 can be the
> "bridge" for MM2 as well.
>
> Ryanne
>
> On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:
>
> > Hi all,
> >
> > We've had a few proposals recently for incompatible changes.  One of them
> > is my KIP-604: Remove ZooKeeper Flags from the Administrative Tools.  The
> > other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> > Controller.  I think it's time to start thinking about Kafka 3.0.
> > Specifically, I think we should move to 3.0 after the 2.6 release.
> >
> > From the perspective of KIP-500, in Kafka 3.x we'd like to make running
> in
> > a ZooKeeper-less mode possible (but not yet the default.)  This is the
> > motivation behind KIP-590 and KIP-604, as well as some of the other KIPs
> > we've done recently.  Since it will take some time to stabilize the new
> > ZooKeeper-free Kafka code, we will hide it behind an option initially.
> > (We'll have a KIP describing this all in detail soon.)
> >
> > What does everyone think about having Kafka 3.0 come up next after 2.6?
> > Are there any other things we should change in the 2.6 -> 3.0 transition?
> >
> > best,
> > Colin
> >
>


-- 
Gwen Shapira
Engineering Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog


Re: [DISCUSS] Kafka 3.0

2020-05-04 Thread Ryanne Dolan
Hey Colin, I think we should wait until after KIP-500's "bridge release" so
there is a clean break from Zookeeper after 3.0. The bridge release by
definition is an attempt to not break anything, so it theoretically doesn't
warrant a major release. If that's not the case (i.e. if a single "bridge
release" turns out to be impractical), we should consider forking 3.0 while
maintaining a line of Zookeeper-dependent Kafka in 2.x. That way 3.x can
evolve dramatically without breaking the 2.x line. In particular, anything
related to removing Zookeeper could land in pre-3.0 while every other
feature targets 2.6.

If you are proposing 2.6 should be the "bridge release", I think this is
premature given Kafka's time-based release schedule. If the bridge features
happen to be merged before 2.6's feature freeze, then sure -- let's make
that the bridge release in retrospect. And if we get all the post-Zookeeper
features merged before 2.7, I'm onboard with naming it "3.0" instead.

That said, we should aim to remove legacy MirrorMaker before 3.0 as well.
I'm happy to drive that additional breaking change. Maybe 2.6 can be the
"bridge" for MM2 as well.

Ryanne

On Mon, May 4, 2020, 5:05 PM Colin McCabe  wrote:

> Hi all,
>
> We've had a few proposals recently for incompatible changes.  One of them
> is my KIP-604: Remove ZooKeeper Flags from the Administrative Tools.  The
> other is Boyang's KIP-590: Redirect ZK Mutation Protocols to the
> Controller.  I think it's time to start thinking about Kafka 3.0.
> Specifically, I think we should move to 3.0 after the 2.6 release.
>
> From the perspective of KIP-500, in Kafka 3.x we'd like to make running in
> a ZooKeeper-less mode possible (but not yet the default.)  This is the
> motivation behind KIP-590 and KIP-604, as well as some of the other KIPs
> we've done recently.  Since it will take some time to stabilize the new
> ZooKeeper-free Kafka code, we will hide it behind an option initially.
> (We'll have a KIP describing this all in detail soon.)
>
> What does everyone think about having Kafka 3.0 come up next after 2.6?
> Are there any other things we should change in the 2.6 -> 3.0 transition?
>
> best,
> Colin
>