Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-10 Thread Almog Gavra
The cut-off line for the builder was to just port whatever is currently in
connect so that we don't break the APIs there. Perhaps it's better to just
introduce the critical methods (partitions/replicas/configs) and have the
one in connect just extend the new one.

On Fri, May 10, 2019 at 3:24 AM Matthias J. Sax 
wrote:

> I minor comment about `.compacted()` method:
>
> It might be better to change to `cleanupPolicy(CleanupPolicy)` and add
> an enum `CleanupPolicy` with fields `DELETE`, `COMPACT`, `DELETE_COMPACT`.
>
> Also, for the cleanup policy, what about retention time vs size?
>
> I am also wondering, what the cut-off line for important configs is,
> that get their own method, vs all other that are set via `config()`.
>
> It seems to be an "random" selection atm.
>
>
> -Matthias
>
> On 5/9/19 6:56 PM, Almog Gavra wrote:
> > Moving discussion on VOTE thread here:
> >
> > Ismael asked:
> >
> >> Adding a Builder seems unrelated to this change. Do we need it? Given
> the
> >> imminent KIP deadline, I'd keep it simple and just have the constructor
> >> with just the name parameter.
> >
> > If we want the flexibility that the builder provides we would need to add
> > three constructors:
> > - no partitions/replicas
> > - just partitions
> > - just replicas
> >
> > I see good use cases for the first two - the third (just replicas) seems
> > less necessary and complicates the API a bit (you have to differentiate
> > NewTopic(int) with NewTopic(short) or something like that). If we're
> happy
> > with a KIP that covers just the first two then I can remove the builder
> to
> > simplify things. Otherwise, I think the builder is an important addition.
> >
> > Thoughts?
> >
> > On Fri, May 3, 2019 at 11:43 AM Randall Hauch  wrote:
> >
> >> I personally like those extra methods, rather than relying upon the
> generic
> >> properties. But I'm fine if others think they should be removed. I'm
> also
> >> fine with not deprecating the Connect version of the builder.
> >>
> >> On Fri, May 3, 2019 at 11:27 AM Almog Gavra  wrote:
> >>
> >>> Ack. KIP updated :) Perhaps instead of deprecating the Connect builder,
> >>> then, we can indeed just subclass it and move some of the less common
> >> build
> >>> methods (e.g. uncleanLeaderElection) there.
> >>>
> >>> On Fri, May 3, 2019 at 11:20 AM Randall Hauch 
> wrote:
> >>>
>  Thanks for updating, Almog. I have a few suggestions specific to the
>  builder:
> 
>  1. The AK pattern for builder classes that are nested is to name the
> >>> class
>  "Builder" and to make it publicly visible. We should follow that
> >> pattern
>  here, too.
>  2. The builder's private constructor makes it impossible to subclass,
>  should we ever want to do that (e.g, in Connect). If we make it
> >> protected
>  or public, then subclassing is easier.
> 
>  On Thu, May 2, 2019 at 9:44 AM Almog Gavra 
> wrote:
> 
> > Thanks for the input Randall. I'm happy adding it natively to
> >> NewTopic
> > instead of introducing more verbosity - updating the KIP to reflect
> >>> this
> > now.
> >
> > On Thu, May 2, 2019 at 9:28 AM Randall Hauch 
> >> wrote:
> >
> >> I wrote the `NewTopicBuilder` in Connect, and it was simply a
>  convenience
> >> to more easily set some of the frequently-used properties and the #
> >>> of
> >> partitions and replicas for the new topic in the same way. An
> >> example
>  is:
> >>
> >> NewTopic topicDescription = TopicAdmin.defineTopic(topic).
> >> compacted().
> >> partitions(1).
> >> replicationFactor(3).
> >> build();
> >>
> >> Arguably it should have been added to clients from the beginning.
> >> So
>  I'm
> >> fine with that being moved to clients, as long as Connect is
> >> changed
> >>> to
> > use
> >> the new clients class. However, even though Connect's
> >>> `NewTopicBuilder`
> > is
> >> in the runtime and technically not part of the public API, things
> >>> like
> > this
> >> still tend to get reused elsewhere. Let's keep the Connect
> >> `NewTopicBuilder` but deprecate it and have it extend the one in
>  clients.
> >> The `TopicAdmin` class in Connect can then refer to the new one in
> > clients.
> >>
> >> The KIP now talks about having a constructor for the builder:
> >>
> >> NewTopic myTopic = new
> >>
> >>
> >
> 
> >>>
> >>
> NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();
> >>
> >> How about adding the builder to the NewTopic class itself:
> >>
> >> NewTopic myTopic =
> >>
> >>
> >
> 
> >>>
> >>
> NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();
> >>
> >> This is a bit shorter, a bit easier to read (no "new New..."), and
> >>> more
> >> discoverable since anyone looking at the NewTopic 

Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-10 Thread Matthias J. Sax
I minor comment about `.compacted()` method:

It might be better to change to `cleanupPolicy(CleanupPolicy)` and add
an enum `CleanupPolicy` with fields `DELETE`, `COMPACT`, `DELETE_COMPACT`.

Also, for the cleanup policy, what about retention time vs size?

I am also wondering, what the cut-off line for important configs is,
that get their own method, vs all other that are set via `config()`.

It seems to be an "random" selection atm.


-Matthias

On 5/9/19 6:56 PM, Almog Gavra wrote:
> Moving discussion on VOTE thread here:
> 
> Ismael asked:
> 
>> Adding a Builder seems unrelated to this change. Do we need it? Given the
>> imminent KIP deadline, I'd keep it simple and just have the constructor
>> with just the name parameter.
> 
> If we want the flexibility that the builder provides we would need to add
> three constructors:
> - no partitions/replicas
> - just partitions
> - just replicas
> 
> I see good use cases for the first two - the third (just replicas) seems
> less necessary and complicates the API a bit (you have to differentiate
> NewTopic(int) with NewTopic(short) or something like that). If we're happy
> with a KIP that covers just the first two then I can remove the builder to
> simplify things. Otherwise, I think the builder is an important addition.
> 
> Thoughts?
> 
> On Fri, May 3, 2019 at 11:43 AM Randall Hauch  wrote:
> 
>> I personally like those extra methods, rather than relying upon the generic
>> properties. But I'm fine if others think they should be removed. I'm also
>> fine with not deprecating the Connect version of the builder.
>>
>> On Fri, May 3, 2019 at 11:27 AM Almog Gavra  wrote:
>>
>>> Ack. KIP updated :) Perhaps instead of deprecating the Connect builder,
>>> then, we can indeed just subclass it and move some of the less common
>> build
>>> methods (e.g. uncleanLeaderElection) there.
>>>
>>> On Fri, May 3, 2019 at 11:20 AM Randall Hauch  wrote:
>>>
 Thanks for updating, Almog. I have a few suggestions specific to the
 builder:

 1. The AK pattern for builder classes that are nested is to name the
>>> class
 "Builder" and to make it publicly visible. We should follow that
>> pattern
 here, too.
 2. The builder's private constructor makes it impossible to subclass,
 should we ever want to do that (e.g, in Connect). If we make it
>> protected
 or public, then subclassing is easier.

 On Thu, May 2, 2019 at 9:44 AM Almog Gavra  wrote:

> Thanks for the input Randall. I'm happy adding it natively to
>> NewTopic
> instead of introducing more verbosity - updating the KIP to reflect
>>> this
> now.
>
> On Thu, May 2, 2019 at 9:28 AM Randall Hauch 
>> wrote:
>
>> I wrote the `NewTopicBuilder` in Connect, and it was simply a
 convenience
>> to more easily set some of the frequently-used properties and the #
>>> of
>> partitions and replicas for the new topic in the same way. An
>> example
 is:
>>
>> NewTopic topicDescription = TopicAdmin.defineTopic(topic).
>> compacted().
>> partitions(1).
>> replicationFactor(3).
>> build();
>>
>> Arguably it should have been added to clients from the beginning.
>> So
 I'm
>> fine with that being moved to clients, as long as Connect is
>> changed
>>> to
> use
>> the new clients class. However, even though Connect's
>>> `NewTopicBuilder`
> is
>> in the runtime and technically not part of the public API, things
>>> like
> this
>> still tend to get reused elsewhere. Let's keep the Connect
>> `NewTopicBuilder` but deprecate it and have it extend the one in
 clients.
>> The `TopicAdmin` class in Connect can then refer to the new one in
> clients.
>>
>> The KIP now talks about having a constructor for the builder:
>>
>> NewTopic myTopic = new
>>
>>
>

>>>
>> NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();
>>
>> How about adding the builder to the NewTopic class itself:
>>
>> NewTopic myTopic =
>>
>>
>

>>>
>> NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();
>>
>> This is a bit shorter, a bit easier to read (no "new New..."), and
>>> more
>> discoverable since anyone looking at the NewTopic source or JavaDoc
 will
>> maybe notice it.
>>
>> Randall
>>
>>
>> On Thu, May 2, 2019 at 8:56 AM Almog Gavra 
>>> wrote:
>>
>>> Sure thing, added more detail to the KIP! To clarify, the plan is
>>> to
> move
>>> an existing API from one package to another (NewTopicBuilder
>> exists
 in
>> the
>>> connect.runtime package) leaving the old in place for
>> compatibility
 and
>>> deprecating it.
>>>
>>> I'm happy to hear thoughts on whether we should (a) move it to
>> the
 same
>>> package in a new module so that 

Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-09 Thread Almog Gavra
Moving discussion on VOTE thread here:

Ismael asked:

> Adding a Builder seems unrelated to this change. Do we need it? Given the
> imminent KIP deadline, I'd keep it simple and just have the constructor
> with just the name parameter.

If we want the flexibility that the builder provides we would need to add
three constructors:
- no partitions/replicas
- just partitions
- just replicas

I see good use cases for the first two - the third (just replicas) seems
less necessary and complicates the API a bit (you have to differentiate
NewTopic(int) with NewTopic(short) or something like that). If we're happy
with a KIP that covers just the first two then I can remove the builder to
simplify things. Otherwise, I think the builder is an important addition.

Thoughts?

On Fri, May 3, 2019 at 11:43 AM Randall Hauch  wrote:

> I personally like those extra methods, rather than relying upon the generic
> properties. But I'm fine if others think they should be removed. I'm also
> fine with not deprecating the Connect version of the builder.
>
> On Fri, May 3, 2019 at 11:27 AM Almog Gavra  wrote:
>
> > Ack. KIP updated :) Perhaps instead of deprecating the Connect builder,
> > then, we can indeed just subclass it and move some of the less common
> build
> > methods (e.g. uncleanLeaderElection) there.
> >
> > On Fri, May 3, 2019 at 11:20 AM Randall Hauch  wrote:
> >
> > > Thanks for updating, Almog. I have a few suggestions specific to the
> > > builder:
> > >
> > > 1. The AK pattern for builder classes that are nested is to name the
> > class
> > > "Builder" and to make it publicly visible. We should follow that
> pattern
> > > here, too.
> > > 2. The builder's private constructor makes it impossible to subclass,
> > > should we ever want to do that (e.g, in Connect). If we make it
> protected
> > > or public, then subclassing is easier.
> > >
> > > On Thu, May 2, 2019 at 9:44 AM Almog Gavra  wrote:
> > >
> > > > Thanks for the input Randall. I'm happy adding it natively to
> NewTopic
> > > > instead of introducing more verbosity - updating the KIP to reflect
> > this
> > > > now.
> > > >
> > > > On Thu, May 2, 2019 at 9:28 AM Randall Hauch 
> wrote:
> > > >
> > > > > I wrote the `NewTopicBuilder` in Connect, and it was simply a
> > > convenience
> > > > > to more easily set some of the frequently-used properties and the #
> > of
> > > > > partitions and replicas for the new topic in the same way. An
> example
> > > is:
> > > > >
> > > > > NewTopic topicDescription = TopicAdmin.defineTopic(topic).
> > > > > compacted().
> > > > > partitions(1).
> > > > > replicationFactor(3).
> > > > > build();
> > > > >
> > > > > Arguably it should have been added to clients from the beginning.
> So
> > > I'm
> > > > > fine with that being moved to clients, as long as Connect is
> changed
> > to
> > > > use
> > > > > the new clients class. However, even though Connect's
> > `NewTopicBuilder`
> > > > is
> > > > > in the runtime and technically not part of the public API, things
> > like
> > > > this
> > > > > still tend to get reused elsewhere. Let's keep the Connect
> > > > > `NewTopicBuilder` but deprecate it and have it extend the one in
> > > clients.
> > > > > The `TopicAdmin` class in Connect can then refer to the new one in
> > > > clients.
> > > > >
> > > > > The KIP now talks about having a constructor for the builder:
> > > > >
> > > > > NewTopic myTopic = new
> > > > >
> > > > >
> > > >
> > >
> >
> NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();
> > > > >
> > > > > How about adding the builder to the NewTopic class itself:
> > > > >
> > > > > NewTopic myTopic =
> > > > >
> > > > >
> > > >
> > >
> >
> NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();
> > > > >
> > > > > This is a bit shorter, a bit easier to read (no "new New..."), and
> > more
> > > > > discoverable since anyone looking at the NewTopic source or JavaDoc
> > > will
> > > > > maybe notice it.
> > > > >
> > > > > Randall
> > > > >
> > > > >
> > > > > On Thu, May 2, 2019 at 8:56 AM Almog Gavra 
> > wrote:
> > > > >
> > > > > > Sure thing, added more detail to the KIP! To clarify, the plan is
> > to
> > > > move
> > > > > > an existing API from one package to another (NewTopicBuilder
> exists
> > > in
> > > > > the
> > > > > > connect.runtime package) leaving the old in place for
> compatibility
> > > and
> > > > > > deprecating it.
> > > > > >
> > > > > > I'm happy to hear thoughts on whether we should (a) move it to
> the
> > > same
> > > > > > package in a new module so that we don't need to deprecate it or
> > (b)
> > > > take
> > > > > > this opportunity to change any of the APIs.
> > > > > >
> > > > > > On Thu, May 2, 2019 at 8:22 AM Ismael Juma 
> > > wrote:
> > > > > >
> > > > > > > If you are adding new API, you need to specify it all in the
> KIP.
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > > > On Thu, May 

Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-03 Thread Randall Hauch
I personally like those extra methods, rather than relying upon the generic
properties. But I'm fine if others think they should be removed. I'm also
fine with not deprecating the Connect version of the builder.

On Fri, May 3, 2019 at 11:27 AM Almog Gavra  wrote:

> Ack. KIP updated :) Perhaps instead of deprecating the Connect builder,
> then, we can indeed just subclass it and move some of the less common build
> methods (e.g. uncleanLeaderElection) there.
>
> On Fri, May 3, 2019 at 11:20 AM Randall Hauch  wrote:
>
> > Thanks for updating, Almog. I have a few suggestions specific to the
> > builder:
> >
> > 1. The AK pattern for builder classes that are nested is to name the
> class
> > "Builder" and to make it publicly visible. We should follow that pattern
> > here, too.
> > 2. The builder's private constructor makes it impossible to subclass,
> > should we ever want to do that (e.g, in Connect). If we make it protected
> > or public, then subclassing is easier.
> >
> > On Thu, May 2, 2019 at 9:44 AM Almog Gavra  wrote:
> >
> > > Thanks for the input Randall. I'm happy adding it natively to NewTopic
> > > instead of introducing more verbosity - updating the KIP to reflect
> this
> > > now.
> > >
> > > On Thu, May 2, 2019 at 9:28 AM Randall Hauch  wrote:
> > >
> > > > I wrote the `NewTopicBuilder` in Connect, and it was simply a
> > convenience
> > > > to more easily set some of the frequently-used properties and the #
> of
> > > > partitions and replicas for the new topic in the same way. An example
> > is:
> > > >
> > > > NewTopic topicDescription = TopicAdmin.defineTopic(topic).
> > > > compacted().
> > > > partitions(1).
> > > > replicationFactor(3).
> > > > build();
> > > >
> > > > Arguably it should have been added to clients from the beginning. So
> > I'm
> > > > fine with that being moved to clients, as long as Connect is changed
> to
> > > use
> > > > the new clients class. However, even though Connect's
> `NewTopicBuilder`
> > > is
> > > > in the runtime and technically not part of the public API, things
> like
> > > this
> > > > still tend to get reused elsewhere. Let's keep the Connect
> > > > `NewTopicBuilder` but deprecate it and have it extend the one in
> > clients.
> > > > The `TopicAdmin` class in Connect can then refer to the new one in
> > > clients.
> > > >
> > > > The KIP now talks about having a constructor for the builder:
> > > >
> > > > NewTopic myTopic = new
> > > >
> > > >
> > >
> >
> NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();
> > > >
> > > > How about adding the builder to the NewTopic class itself:
> > > >
> > > > NewTopic myTopic =
> > > >
> > > >
> > >
> >
> NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();
> > > >
> > > > This is a bit shorter, a bit easier to read (no "new New..."), and
> more
> > > > discoverable since anyone looking at the NewTopic source or JavaDoc
> > will
> > > > maybe notice it.
> > > >
> > > > Randall
> > > >
> > > >
> > > > On Thu, May 2, 2019 at 8:56 AM Almog Gavra 
> wrote:
> > > >
> > > > > Sure thing, added more detail to the KIP! To clarify, the plan is
> to
> > > move
> > > > > an existing API from one package to another (NewTopicBuilder exists
> > in
> > > > the
> > > > > connect.runtime package) leaving the old in place for compatibility
> > and
> > > > > deprecating it.
> > > > >
> > > > > I'm happy to hear thoughts on whether we should (a) move it to the
> > same
> > > > > package in a new module so that we don't need to deprecate it or
> (b)
> > > take
> > > > > this opportunity to change any of the APIs.
> > > > >
> > > > > On Thu, May 2, 2019 at 8:22 AM Ismael Juma 
> > wrote:
> > > > >
> > > > > > If you are adding new API, you need to specify it all in the KIP.
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > On Thu, May 2, 2019, 8:04 AM Almog Gavra 
> > wrote:
> > > > > >
> > > > > > > I think that sounds reasonable - I updated the KIP and I will
> > > remove
> > > > > the
> > > > > > > constructor that takes in only partitions.
> > > > > > >
> > > > > > > On Thu, May 2, 2019 at 4:44 AM Andy Coates 
> > > > wrote:
> > > > > > >
> > > > > > > > Rather than adding overloaded constructors, which can lead to
> > API
> > > > > > bloat,
> > > > > > > > how about using a builder pattern?
> > > > > > > >
> > > > > > > > I see it’s already got some constructor overloading, but we
> > could
> > > > > add a
> > > > > > > > single new constructor that takes just the name, and support
> > > > > everything
> > > > > > > > else being set via builder methods.
> > > > > > > >
> > > > > > > > This would result in a better long term api as the number of
> > > > options
> > > > > > > > increases.
> > > > > > > >
> > > > > > > > Sent from my iPhone
> > > > > > > >
> > > > > > > > > On 30 Apr 2019, at 16:28, Almog Gavra 
> > > > wrote:
> > > > > > > > >
> > > > > > > > > Hello Everyone,
> > > > > > > > >
> > 

Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-03 Thread Almog Gavra
Ack. KIP updated :) Perhaps instead of deprecating the Connect builder,
then, we can indeed just subclass it and move some of the less common build
methods (e.g. uncleanLeaderElection) there.

On Fri, May 3, 2019 at 11:20 AM Randall Hauch  wrote:

> Thanks for updating, Almog. I have a few suggestions specific to the
> builder:
>
> 1. The AK pattern for builder classes that are nested is to name the class
> "Builder" and to make it publicly visible. We should follow that pattern
> here, too.
> 2. The builder's private constructor makes it impossible to subclass,
> should we ever want to do that (e.g, in Connect). If we make it protected
> or public, then subclassing is easier.
>
> On Thu, May 2, 2019 at 9:44 AM Almog Gavra  wrote:
>
> > Thanks for the input Randall. I'm happy adding it natively to NewTopic
> > instead of introducing more verbosity - updating the KIP to reflect this
> > now.
> >
> > On Thu, May 2, 2019 at 9:28 AM Randall Hauch  wrote:
> >
> > > I wrote the `NewTopicBuilder` in Connect, and it was simply a
> convenience
> > > to more easily set some of the frequently-used properties and the # of
> > > partitions and replicas for the new topic in the same way. An example
> is:
> > >
> > > NewTopic topicDescription = TopicAdmin.defineTopic(topic).
> > > compacted().
> > > partitions(1).
> > > replicationFactor(3).
> > > build();
> > >
> > > Arguably it should have been added to clients from the beginning. So
> I'm
> > > fine with that being moved to clients, as long as Connect is changed to
> > use
> > > the new clients class. However, even though Connect's `NewTopicBuilder`
> > is
> > > in the runtime and technically not part of the public API, things like
> > this
> > > still tend to get reused elsewhere. Let's keep the Connect
> > > `NewTopicBuilder` but deprecate it and have it extend the one in
> clients.
> > > The `TopicAdmin` class in Connect can then refer to the new one in
> > clients.
> > >
> > > The KIP now talks about having a constructor for the builder:
> > >
> > > NewTopic myTopic = new
> > >
> > >
> >
> NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();
> > >
> > > How about adding the builder to the NewTopic class itself:
> > >
> > > NewTopic myTopic =
> > >
> > >
> >
> NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();
> > >
> > > This is a bit shorter, a bit easier to read (no "new New..."), and more
> > > discoverable since anyone looking at the NewTopic source or JavaDoc
> will
> > > maybe notice it.
> > >
> > > Randall
> > >
> > >
> > > On Thu, May 2, 2019 at 8:56 AM Almog Gavra  wrote:
> > >
> > > > Sure thing, added more detail to the KIP! To clarify, the plan is to
> > move
> > > > an existing API from one package to another (NewTopicBuilder exists
> in
> > > the
> > > > connect.runtime package) leaving the old in place for compatibility
> and
> > > > deprecating it.
> > > >
> > > > I'm happy to hear thoughts on whether we should (a) move it to the
> same
> > > > package in a new module so that we don't need to deprecate it or (b)
> > take
> > > > this opportunity to change any of the APIs.
> > > >
> > > > On Thu, May 2, 2019 at 8:22 AM Ismael Juma 
> wrote:
> > > >
> > > > > If you are adding new API, you need to specify it all in the KIP.
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Thu, May 2, 2019, 8:04 AM Almog Gavra 
> wrote:
> > > > >
> > > > > > I think that sounds reasonable - I updated the KIP and I will
> > remove
> > > > the
> > > > > > constructor that takes in only partitions.
> > > > > >
> > > > > > On Thu, May 2, 2019 at 4:44 AM Andy Coates 
> > > wrote:
> > > > > >
> > > > > > > Rather than adding overloaded constructors, which can lead to
> API
> > > > > bloat,
> > > > > > > how about using a builder pattern?
> > > > > > >
> > > > > > > I see it’s already got some constructor overloading, but we
> could
> > > > add a
> > > > > > > single new constructor that takes just the name, and support
> > > > everything
> > > > > > > else being set via builder methods.
> > > > > > >
> > > > > > > This would result in a better long term api as the number of
> > > options
> > > > > > > increases.
> > > > > > >
> > > > > > > Sent from my iPhone
> > > > > > >
> > > > > > > > On 30 Apr 2019, at 16:28, Almog Gavra 
> > > wrote:
> > > > > > > >
> > > > > > > > Hello Everyone,
> > > > > > > >
> > > > > > > > I'd like to start a discussion on KIP-464:
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > > > > > > >
> > > > > > > > It's about allowing users of the AdminClient to supply only a
> > > > > partition
> > > > > > > > count and to use the default replication factor configured by
> > the
> > > > > kafka
> > > > > > > > cluster. Happy to receive any and all feedback!
> > > > > > > >
> > > > > 

Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-03 Thread Randall Hauch
Thanks for updating, Almog. I have a few suggestions specific to the
builder:

1. The AK pattern for builder classes that are nested is to name the class
"Builder" and to make it publicly visible. We should follow that pattern
here, too.
2. The builder's private constructor makes it impossible to subclass,
should we ever want to do that (e.g, in Connect). If we make it protected
or public, then subclassing is easier.

On Thu, May 2, 2019 at 9:44 AM Almog Gavra  wrote:

> Thanks for the input Randall. I'm happy adding it natively to NewTopic
> instead of introducing more verbosity - updating the KIP to reflect this
> now.
>
> On Thu, May 2, 2019 at 9:28 AM Randall Hauch  wrote:
>
> > I wrote the `NewTopicBuilder` in Connect, and it was simply a convenience
> > to more easily set some of the frequently-used properties and the # of
> > partitions and replicas for the new topic in the same way. An example is:
> >
> > NewTopic topicDescription = TopicAdmin.defineTopic(topic).
> > compacted().
> > partitions(1).
> > replicationFactor(3).
> > build();
> >
> > Arguably it should have been added to clients from the beginning. So I'm
> > fine with that being moved to clients, as long as Connect is changed to
> use
> > the new clients class. However, even though Connect's `NewTopicBuilder`
> is
> > in the runtime and technically not part of the public API, things like
> this
> > still tend to get reused elsewhere. Let's keep the Connect
> > `NewTopicBuilder` but deprecate it and have it extend the one in clients.
> > The `TopicAdmin` class in Connect can then refer to the new one in
> clients.
> >
> > The KIP now talks about having a constructor for the builder:
> >
> > NewTopic myTopic = new
> >
> >
> NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();
> >
> > How about adding the builder to the NewTopic class itself:
> >
> > NewTopic myTopic =
> >
> >
> NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();
> >
> > This is a bit shorter, a bit easier to read (no "new New..."), and more
> > discoverable since anyone looking at the NewTopic source or JavaDoc will
> > maybe notice it.
> >
> > Randall
> >
> >
> > On Thu, May 2, 2019 at 8:56 AM Almog Gavra  wrote:
> >
> > > Sure thing, added more detail to the KIP! To clarify, the plan is to
> move
> > > an existing API from one package to another (NewTopicBuilder exists in
> > the
> > > connect.runtime package) leaving the old in place for compatibility and
> > > deprecating it.
> > >
> > > I'm happy to hear thoughts on whether we should (a) move it to the same
> > > package in a new module so that we don't need to deprecate it or (b)
> take
> > > this opportunity to change any of the APIs.
> > >
> > > On Thu, May 2, 2019 at 8:22 AM Ismael Juma  wrote:
> > >
> > > > If you are adding new API, you need to specify it all in the KIP.
> > > >
> > > > Ismael
> > > >
> > > > On Thu, May 2, 2019, 8:04 AM Almog Gavra  wrote:
> > > >
> > > > > I think that sounds reasonable - I updated the KIP and I will
> remove
> > > the
> > > > > constructor that takes in only partitions.
> > > > >
> > > > > On Thu, May 2, 2019 at 4:44 AM Andy Coates 
> > wrote:
> > > > >
> > > > > > Rather than adding overloaded constructors, which can lead to API
> > > > bloat,
> > > > > > how about using a builder pattern?
> > > > > >
> > > > > > I see it’s already got some constructor overloading, but we could
> > > add a
> > > > > > single new constructor that takes just the name, and support
> > > everything
> > > > > > else being set via builder methods.
> > > > > >
> > > > > > This would result in a better long term api as the number of
> > options
> > > > > > increases.
> > > > > >
> > > > > > Sent from my iPhone
> > > > > >
> > > > > > > On 30 Apr 2019, at 16:28, Almog Gavra 
> > wrote:
> > > > > > >
> > > > > > > Hello Everyone,
> > > > > > >
> > > > > > > I'd like to start a discussion on KIP-464:
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > > > > > >
> > > > > > > It's about allowing users of the AdminClient to supply only a
> > > > partition
> > > > > > > count and to use the default replication factor configured by
> the
> > > > kafka
> > > > > > > cluster. Happy to receive any and all feedback!
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Almog
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-02 Thread Almog Gavra
Thanks for the input Randall. I'm happy adding it natively to NewTopic
instead of introducing more verbosity - updating the KIP to reflect this
now.

On Thu, May 2, 2019 at 9:28 AM Randall Hauch  wrote:

> I wrote the `NewTopicBuilder` in Connect, and it was simply a convenience
> to more easily set some of the frequently-used properties and the # of
> partitions and replicas for the new topic in the same way. An example is:
>
> NewTopic topicDescription = TopicAdmin.defineTopic(topic).
> compacted().
> partitions(1).
> replicationFactor(3).
> build();
>
> Arguably it should have been added to clients from the beginning. So I'm
> fine with that being moved to clients, as long as Connect is changed to use
> the new clients class. However, even though Connect's `NewTopicBuilder` is
> in the runtime and technically not part of the public API, things like this
> still tend to get reused elsewhere. Let's keep the Connect
> `NewTopicBuilder` but deprecate it and have it extend the one in clients.
> The `TopicAdmin` class in Connect can then refer to the new one in clients.
>
> The KIP now talks about having a constructor for the builder:
>
> NewTopic myTopic = new
>
> NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();
>
> How about adding the builder to the NewTopic class itself:
>
> NewTopic myTopic =
>
> NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();
>
> This is a bit shorter, a bit easier to read (no "new New..."), and more
> discoverable since anyone looking at the NewTopic source or JavaDoc will
> maybe notice it.
>
> Randall
>
>
> On Thu, May 2, 2019 at 8:56 AM Almog Gavra  wrote:
>
> > Sure thing, added more detail to the KIP! To clarify, the plan is to move
> > an existing API from one package to another (NewTopicBuilder exists in
> the
> > connect.runtime package) leaving the old in place for compatibility and
> > deprecating it.
> >
> > I'm happy to hear thoughts on whether we should (a) move it to the same
> > package in a new module so that we don't need to deprecate it or (b) take
> > this opportunity to change any of the APIs.
> >
> > On Thu, May 2, 2019 at 8:22 AM Ismael Juma  wrote:
> >
> > > If you are adding new API, you need to specify it all in the KIP.
> > >
> > > Ismael
> > >
> > > On Thu, May 2, 2019, 8:04 AM Almog Gavra  wrote:
> > >
> > > > I think that sounds reasonable - I updated the KIP and I will remove
> > the
> > > > constructor that takes in only partitions.
> > > >
> > > > On Thu, May 2, 2019 at 4:44 AM Andy Coates 
> wrote:
> > > >
> > > > > Rather than adding overloaded constructors, which can lead to API
> > > bloat,
> > > > > how about using a builder pattern?
> > > > >
> > > > > I see it’s already got some constructor overloading, but we could
> > add a
> > > > > single new constructor that takes just the name, and support
> > everything
> > > > > else being set via builder methods.
> > > > >
> > > > > This would result in a better long term api as the number of
> options
> > > > > increases.
> > > > >
> > > > > Sent from my iPhone
> > > > >
> > > > > > On 30 Apr 2019, at 16:28, Almog Gavra 
> wrote:
> > > > > >
> > > > > > Hello Everyone,
> > > > > >
> > > > > > I'd like to start a discussion on KIP-464:
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > > > > >
> > > > > > It's about allowing users of the AdminClient to supply only a
> > > partition
> > > > > > count and to use the default replication factor configured by the
> > > kafka
> > > > > > cluster. Happy to receive any and all feedback!
> > > > > >
> > > > > > Cheers,
> > > > > > Almog
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-02 Thread Randall Hauch
I wrote the `NewTopicBuilder` in Connect, and it was simply a convenience
to more easily set some of the frequently-used properties and the # of
partitions and replicas for the new topic in the same way. An example is:

NewTopic topicDescription = TopicAdmin.defineTopic(topic).
compacted().
partitions(1).
replicationFactor(3).
build();

Arguably it should have been added to clients from the beginning. So I'm
fine with that being moved to clients, as long as Connect is changed to use
the new clients class. However, even though Connect's `NewTopicBuilder` is
in the runtime and technically not part of the public API, things like this
still tend to get reused elsewhere. Let's keep the Connect
`NewTopicBuilder` but deprecate it and have it extend the one in clients.
The `TopicAdmin` class in Connect can then refer to the new one in clients.

The KIP now talks about having a constructor for the builder:

NewTopic myTopic = new
NewTopicBuilder(name).compacted().partitions(1).replicationFactor(3).build();

How about adding the builder to the NewTopic class itself:

NewTopic myTopic =
NewTopic.build(name).compacted().partitions(1).replicationFactor(3).build();

This is a bit shorter, a bit easier to read (no "new New..."), and more
discoverable since anyone looking at the NewTopic source or JavaDoc will
maybe notice it.

Randall


On Thu, May 2, 2019 at 8:56 AM Almog Gavra  wrote:

> Sure thing, added more detail to the KIP! To clarify, the plan is to move
> an existing API from one package to another (NewTopicBuilder exists in the
> connect.runtime package) leaving the old in place for compatibility and
> deprecating it.
>
> I'm happy to hear thoughts on whether we should (a) move it to the same
> package in a new module so that we don't need to deprecate it or (b) take
> this opportunity to change any of the APIs.
>
> On Thu, May 2, 2019 at 8:22 AM Ismael Juma  wrote:
>
> > If you are adding new API, you need to specify it all in the KIP.
> >
> > Ismael
> >
> > On Thu, May 2, 2019, 8:04 AM Almog Gavra  wrote:
> >
> > > I think that sounds reasonable - I updated the KIP and I will remove
> the
> > > constructor that takes in only partitions.
> > >
> > > On Thu, May 2, 2019 at 4:44 AM Andy Coates  wrote:
> > >
> > > > Rather than adding overloaded constructors, which can lead to API
> > bloat,
> > > > how about using a builder pattern?
> > > >
> > > > I see it’s already got some constructor overloading, but we could
> add a
> > > > single new constructor that takes just the name, and support
> everything
> > > > else being set via builder methods.
> > > >
> > > > This would result in a better long term api as the number of options
> > > > increases.
> > > >
> > > > Sent from my iPhone
> > > >
> > > > > On 30 Apr 2019, at 16:28, Almog Gavra  wrote:
> > > > >
> > > > > Hello Everyone,
> > > > >
> > > > > I'd like to start a discussion on KIP-464:
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > > > >
> > > > > It's about allowing users of the AdminClient to supply only a
> > partition
> > > > > count and to use the default replication factor configured by the
> > kafka
> > > > > cluster. Happy to receive any and all feedback!
> > > > >
> > > > > Cheers,
> > > > > Almog
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-02 Thread Almog Gavra
Sure thing, added more detail to the KIP! To clarify, the plan is to move
an existing API from one package to another (NewTopicBuilder exists in the
connect.runtime package) leaving the old in place for compatibility and
deprecating it.

I'm happy to hear thoughts on whether we should (a) move it to the same
package in a new module so that we don't need to deprecate it or (b) take
this opportunity to change any of the APIs.

On Thu, May 2, 2019 at 8:22 AM Ismael Juma  wrote:

> If you are adding new API, you need to specify it all in the KIP.
>
> Ismael
>
> On Thu, May 2, 2019, 8:04 AM Almog Gavra  wrote:
>
> > I think that sounds reasonable - I updated the KIP and I will remove the
> > constructor that takes in only partitions.
> >
> > On Thu, May 2, 2019 at 4:44 AM Andy Coates  wrote:
> >
> > > Rather than adding overloaded constructors, which can lead to API
> bloat,
> > > how about using a builder pattern?
> > >
> > > I see it’s already got some constructor overloading, but we could add a
> > > single new constructor that takes just the name, and support everything
> > > else being set via builder methods.
> > >
> > > This would result in a better long term api as the number of options
> > > increases.
> > >
> > > Sent from my iPhone
> > >
> > > > On 30 Apr 2019, at 16:28, Almog Gavra  wrote:
> > > >
> > > > Hello Everyone,
> > > >
> > > > I'd like to start a discussion on KIP-464:
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > > >
> > > > It's about allowing users of the AdminClient to supply only a
> partition
> > > > count and to use the default replication factor configured by the
> kafka
> > > > cluster. Happy to receive any and all feedback!
> > > >
> > > > Cheers,
> > > > Almog
> > >
> >
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-02 Thread Ismael Juma
If you are adding new API, you need to specify it all in the KIP.

Ismael

On Thu, May 2, 2019, 8:04 AM Almog Gavra  wrote:

> I think that sounds reasonable - I updated the KIP and I will remove the
> constructor that takes in only partitions.
>
> On Thu, May 2, 2019 at 4:44 AM Andy Coates  wrote:
>
> > Rather than adding overloaded constructors, which can lead to API bloat,
> > how about using a builder pattern?
> >
> > I see it’s already got some constructor overloading, but we could add a
> > single new constructor that takes just the name, and support everything
> > else being set via builder methods.
> >
> > This would result in a better long term api as the number of options
> > increases.
> >
> > Sent from my iPhone
> >
> > > On 30 Apr 2019, at 16:28, Almog Gavra  wrote:
> > >
> > > Hello Everyone,
> > >
> > > I'd like to start a discussion on KIP-464:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > >
> > > It's about allowing users of the AdminClient to supply only a partition
> > > count and to use the default replication factor configured by the kafka
> > > cluster. Happy to receive any and all feedback!
> > >
> > > Cheers,
> > > Almog
> >
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-02 Thread Almog Gavra
I think that sounds reasonable - I updated the KIP and I will remove the
constructor that takes in only partitions.

On Thu, May 2, 2019 at 4:44 AM Andy Coates  wrote:

> Rather than adding overloaded constructors, which can lead to API bloat,
> how about using a builder pattern?
>
> I see it’s already got some constructor overloading, but we could add a
> single new constructor that takes just the name, and support everything
> else being set via builder methods.
>
> This would result in a better long term api as the number of options
> increases.
>
> Sent from my iPhone
>
> > On 30 Apr 2019, at 16:28, Almog Gavra  wrote:
> >
> > Hello Everyone,
> >
> > I'd like to start a discussion on KIP-464:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> >
> > It's about allowing users of the AdminClient to supply only a partition
> > count and to use the default replication factor configured by the kafka
> > cluster. Happy to receive any and all feedback!
> >
> > Cheers,
> > Almog
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-02 Thread Andy Coates
Rather than adding overloaded constructors, which can lead to API bloat, how 
about using a builder pattern?

I see it’s already got some constructor overloading, but we could add a single 
new constructor that takes just the name, and support everything else being set 
via builder methods.

This would result in a better long term api as the number of options increases.

Sent from my iPhone

> On 30 Apr 2019, at 16:28, Almog Gavra  wrote:
> 
> Hello Everyone,
> 
> I'd like to start a discussion on KIP-464:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> 
> It's about allowing users of the AdminClient to supply only a partition
> count and to use the default replication factor configured by the kafka
> cluster. Happy to receive any and all feedback!
> 
> Cheers,
> Almog


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-04-30 Thread Almog Gavra
"lack of default doesn't force people to think - it gets them to pick a
random number" - I suppose if I'm being pragmatic this rings very true to
my experience as well... better a cluster administrator sets a sane default
than having users choose random numbers!

Consider me convinced - I'll update the KIP to reflect this: if replicas,
partitions and manual assignments are all missing, the cluster defaults
will be used.

On Tue, Apr 30, 2019 at 11:36 AM Gwen Shapira  wrote:

> Changing number of partitions is complicated in some use-cases and easy in
> other cases (when you use Kafka as a big pipe of events on the way to
> another system).
> I like making easy things easy and complex things complicated. Having
> defaults for both will allow the easy cases to be even easier.
>
> In my experience lack of default doesn't force people to think - it gets
> them to pick a random number... (not sure there is any configuration that
> can get anyone to think, unfortunately).
>
> On Tue, Apr 30, 2019 at 10:22 AM Almog Gavra  wrote:
>
> > I have a preference toward requiring specifying partitions per topic, but
> > I'm happy to be convinced otherwise. Changing replication factor after
> the
> > fact is easy, but changing partitions is complicated since historical
> state
> > gets messed up, so it could be beneficial to force clients to think about
> > it up front. Furthermore, I see partitioning as a function on the scale
> of
> > the data while replication is a function of cluster capacity. Thoughts?
> >
> > On Tue, Apr 30, 2019 at 8:58 AM Ismael Juma  wrote:
> >
> > > Thanks for the KIP, Almog. This is a good change. I think we should
> also
> > > allow the partition count broker default to be used (the one used for
> > auto
> > > topic creation).
> > >
> > > Ismael
> > >
> > > On Tue, Apr 30, 2019, 8:39 AM Almog Gavra  wrote:
> > >
> > > > Hello Everyone,
> > > >
> > > > I'd like to start a discussion on KIP-464:
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > > >
> > > > It's about allowing users of the AdminClient to supply only a
> partition
> > > > count and to use the default replication factor configured by the
> kafka
> > > > cluster. Happy to receive any and all feedback!
> > > >
> > > > Cheers,
> > > > Almog
> > > >
> > >
> >
>
>
> --
> *Gwen Shapira*
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter  | blog
> 
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-04-30 Thread Gwen Shapira
Changing number of partitions is complicated in some use-cases and easy in
other cases (when you use Kafka as a big pipe of events on the way to
another system).
I like making easy things easy and complex things complicated. Having
defaults for both will allow the easy cases to be even easier.

In my experience lack of default doesn't force people to think - it gets
them to pick a random number... (not sure there is any configuration that
can get anyone to think, unfortunately).

On Tue, Apr 30, 2019 at 10:22 AM Almog Gavra  wrote:

> I have a preference toward requiring specifying partitions per topic, but
> I'm happy to be convinced otherwise. Changing replication factor after the
> fact is easy, but changing partitions is complicated since historical state
> gets messed up, so it could be beneficial to force clients to think about
> it up front. Furthermore, I see partitioning as a function on the scale of
> the data while replication is a function of cluster capacity. Thoughts?
>
> On Tue, Apr 30, 2019 at 8:58 AM Ismael Juma  wrote:
>
> > Thanks for the KIP, Almog. This is a good change. I think we should also
> > allow the partition count broker default to be used (the one used for
> auto
> > topic creation).
> >
> > Ismael
> >
> > On Tue, Apr 30, 2019, 8:39 AM Almog Gavra  wrote:
> >
> > > Hello Everyone,
> > >
> > > I'd like to start a discussion on KIP-464:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> > >
> > > It's about allowing users of the AdminClient to supply only a partition
> > > count and to use the default replication factor configured by the kafka
> > > cluster. Happy to receive any and all feedback!
> > >
> > > Cheers,
> > > Almog
> > >
> >
>


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



Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-04-30 Thread Almog Gavra
I have a preference toward requiring specifying partitions per topic, but
I'm happy to be convinced otherwise. Changing replication factor after the
fact is easy, but changing partitions is complicated since historical state
gets messed up, so it could be beneficial to force clients to think about
it up front. Furthermore, I see partitioning as a function on the scale of
the data while replication is a function of cluster capacity. Thoughts?

On Tue, Apr 30, 2019 at 8:58 AM Ismael Juma  wrote:

> Thanks for the KIP, Almog. This is a good change. I think we should also
> allow the partition count broker default to be used (the one used for auto
> topic creation).
>
> Ismael
>
> On Tue, Apr 30, 2019, 8:39 AM Almog Gavra  wrote:
>
> > Hello Everyone,
> >
> > I'd like to start a discussion on KIP-464:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
> >
> > It's about allowing users of the AdminClient to supply only a partition
> > count and to use the default replication factor configured by the kafka
> > cluster. Happy to receive any and all feedback!
> >
> > Cheers,
> > Almog
> >
>


Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-04-30 Thread Ismael Juma
Thanks for the KIP, Almog. This is a good change. I think we should also
allow the partition count broker default to be used (the one used for auto
topic creation).

Ismael

On Tue, Apr 30, 2019, 8:39 AM Almog Gavra  wrote:

> Hello Everyone,
>
> I'd like to start a discussion on KIP-464:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-464%3A+Default+Replication+Factor+for+AdminClient%23createTopic
>
> It's about allowing users of the AdminClient to supply only a partition
> count and to use the default replication factor configured by the kafka
> cluster. Happy to receive any and all feedback!
>
> Cheers,
> Almog
>