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 <al...@confluent.io> 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 <rha...@gmail.com> 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 <al...@confluent.io> 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 <isma...@gmail.com> 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 <al...@confluent.io> 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 <a...@confluent.io>
> > 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 <al...@confluent.io>
> > 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
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to