Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-17 Thread Colin McCabe
On Tue, Mar 14, 2017, at 18:43, Becket Qin wrote:
> Hi Colin,
> Thanks for the reply. Please see comments inline.
> 
> On Tue, Mar 14, 2017 at 5:30 PM, Colin McCabe  wrote:
> > I'm not sure if there is a benefit to removing createTopic and
> > deleteTopic.  It seems like users who want to create or delete just one
> > topic would have to add more boilerplate.  I'm curious what others think
> > here.
> 
> I think many users actually consume from only one topic. I had the same
> doubt about whether a bathing only API would cause problem for the users
> when we changed the consumer API, but it seem to be fine. For users who
> wants to only operate on a single topic they can do something like:
> 
> createTopics(Collcetions.singleton("topic"));
> 
> This looks clean enough.

Hi Becket,

I don't feel very strongly about this.  But so far I don't see a strong
disadvantage to having both the singular and plural forms.  Further
extensions to the API will all be through the CreateTopicsOptions and
NewTopic classes, so the number should stay fixed at 4 rather than
growing.

> 
> >
> > >
> > > 2. NewTopic.setConfigs() is a little weird, can it just be part of the
> > > constructor? Any specific reason to change the configs after the creation
> > > of a NewTopic instance?
> >
> > I guess the thinking here was that the constructor contains the fields
> > that are mandatory (name and partition assignments / strategy), but
> > setting a non-default config is optional.
> >
> Can we just have two constructors? I understand that sometimes the
> exploding number of constructors is annoying, but having two constructors
> here seems acceptable. It would avoid introducing the mutability which
> seems not expected.

We already have two constructors, so we would have four if we added
another degree of freedom.  But I suppose we can do it, to preserve
immutability.  I will change the proposal.

best,
Colin

> 
> >
> > cheers,
> > Colin
> >
> >
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > > On Tue, Mar 14, 2017 at 10:15 AM, Ismael Juma  wrote:
> > >
> > > > Hi Gwen,
> > > >
> > > > I agree with you that it's unfortunate that we now have 4 methods per
> > > > operation (including the batch ones). I personally still prefer the
> > option
> > > > of using a single class to represent the parameters of each method
> > (i.e. 1
> > > > batch and 1 non-batch method per operation). I would follow a very
> > simple
> > > > and boring pattern for each operation:
> > > >
> > > > CreateTopicResults createTopic(CreateTopicRequest request);
> > > > CreateTopicsResults createTopics(CreateTopicsRequest request);
> > > >
> > > > This is simple to understand, easy to keep consistent as we add
> > methods and
> > > > easy to evolve. A downside is that it's a bit more verbose than the
> > current
> > > > proposal.
> > > >
> > > > Regarding LogConfig, we can't use it as it's in the core module (and in
> > > > Scala).
> > > >
> > > > I'm not too sure about the Unstable point. A couple of points:
> > > >
> > > > 1. A major reason why we are adding the AdminClient is so that people
> > don't
> > > > have to deal with incompatible changes. So, unless we truly have to, I
> > > > don't think we'll be making incompatible changes after the release.
> > > >
> > > > 2. If we make changes to unstable code, it still has to go through the
> > KIP
> > > > process. If they're minor changes in this release cycle, we can simply
> > send
> > > > an email to the discuss thread, but if it's a major change, then it
> > would
> > > > require a new round of voting (or a new KIP).
> > > >
> > > > Anyway, I think we're close on this one, but it would be good to agree
> > on
> > > > the general pattern so that we can easily add more methods.
> > > >
> > > > Ismael
> > > >
> > > > On Tue, Mar 14, 2017 at 5:29 AM, Gwen Shapira 
> > wrote:
> > > >
> > > > > I'm torn between my desire to get this in already and the fact that
> > parts
> > > > > of the API feel a bit alien to Kafka.
> > > > >
> > > > > I will resolve my difficulties by giving my feedback here and then
> > going
> > > > to
> > > > > vote +1 on the vote thread.
> > > > > Colin can choose whether to address my concerns now or use his
> > "unstable"
> > > > > option to wait and see...
> > > > >
> > > > > My main concern is the RequestOptions objects... It was discussed
> > > > earlier a
> > > > > bit, but I'm not sure my particular concerns were addressed. I see
> > the
> > > > > following issues with it:
> > > > > * They double the number of methods we have
> > > > > * They are pretty similar, so it isn't clear why we need all those
> > > > > different objects
> > > > > * All our other APIs specify timeouts either in method calls
> > directly or
> > > > in
> > > > > the configuration for the entire object.
> > > > >
> > > > > We also typically don't use methods that start with "set", but this
> > is
> > > > > minor.
> > > > >
> > > > > The 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-14 Thread Becket Qin
Hi Colin,

Thanks for the reply. Please see comments inline.

On Tue, Mar 14, 2017 at 5:30 PM, Colin McCabe  wrote:

> On Tue, Mar 14, 2017, at 13:36, Becket Qin wrote:
> > The interface looks good overall. Thanks for the much needed work Colin.
>
> Thanks, Becket.
>
> > Just a few comments:
> >
> > 1. I agree with Gwen that it is a little unfortunate we have to double
> > the
> > methods for batching interface. In the new consumer interface we only
> > have
> > batch interface and it seems to be working fine. Maybe we can do the same
> > here by just remove all the non-batching interface.
>
> Hmm.  The consumer seems like a different use-case, because pretty much
> everyone consumes more than one record from Kafka over the lifespan of
> their consumer, but a significant fraction of admin client users may
> only want to delete or create one topic.


> I'm not sure if there is a benefit to removing createTopic and
> deleteTopic.  It seems like users who want to create or delete just one
> topic would have to add more boilerplate.  I'm curious what others think
> here.

I think many users actually consume from only one topic. I had the same
doubt about whether a bathing only API would cause problem for the users
when we changed the consumer API, but it seem to be fine. For users who
wants to only operate on a single topic they can do something like:

createTopics(Collcetions.singleton("topic"));

This looks clean enough.

>
> >
> > 2. NewTopic.setConfigs() is a little weird, can it just be part of the
> > constructor? Any specific reason to change the configs after the creation
> > of a NewTopic instance?
>
> I guess the thinking here was that the constructor contains the fields
> that are mandatory (name and partition assignments / strategy), but
> setting a non-default config is optional.
>
Can we just have two constructors? I understand that sometimes the
exploding number of constructors is annoying, but having two constructors
here seems acceptable. It would avoid introducing the mutability which
seems not expected.

>
> cheers,
> Colin
>
>
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Tue, Mar 14, 2017 at 10:15 AM, Ismael Juma  wrote:
> >
> > > Hi Gwen,
> > >
> > > I agree with you that it's unfortunate that we now have 4 methods per
> > > operation (including the batch ones). I personally still prefer the
> option
> > > of using a single class to represent the parameters of each method
> (i.e. 1
> > > batch and 1 non-batch method per operation). I would follow a very
> simple
> > > and boring pattern for each operation:
> > >
> > > CreateTopicResults createTopic(CreateTopicRequest request);
> > > CreateTopicsResults createTopics(CreateTopicsRequest request);
> > >
> > > This is simple to understand, easy to keep consistent as we add
> methods and
> > > easy to evolve. A downside is that it's a bit more verbose than the
> current
> > > proposal.
> > >
> > > Regarding LogConfig, we can't use it as it's in the core module (and in
> > > Scala).
> > >
> > > I'm not too sure about the Unstable point. A couple of points:
> > >
> > > 1. A major reason why we are adding the AdminClient is so that people
> don't
> > > have to deal with incompatible changes. So, unless we truly have to, I
> > > don't think we'll be making incompatible changes after the release.
> > >
> > > 2. If we make changes to unstable code, it still has to go through the
> KIP
> > > process. If they're minor changes in this release cycle, we can simply
> send
> > > an email to the discuss thread, but if it's a major change, then it
> would
> > > require a new round of voting (or a new KIP).
> > >
> > > Anyway, I think we're close on this one, but it would be good to agree
> on
> > > the general pattern so that we can easily add more methods.
> > >
> > > Ismael
> > >
> > > On Tue, Mar 14, 2017 at 5:29 AM, Gwen Shapira 
> wrote:
> > >
> > > > I'm torn between my desire to get this in already and the fact that
> parts
> > > > of the API feel a bit alien to Kafka.
> > > >
> > > > I will resolve my difficulties by giving my feedback here and then
> going
> > > to
> > > > vote +1 on the vote thread.
> > > > Colin can choose whether to address my concerns now or use his
> "unstable"
> > > > option to wait and see...
> > > >
> > > > My main concern is the RequestOptions objects... It was discussed
> > > earlier a
> > > > bit, but I'm not sure my particular concerns were addressed. I see
> the
> > > > following issues with it:
> > > > * They double the number of methods we have
> > > > * They are pretty similar, so it isn't clear why we need all those
> > > > different objects
> > > > * All our other APIs specify timeouts either in method calls
> directly or
> > > in
> > > > the configuration for the entire object.
> > > >
> > > > We also typically don't use methods that start with "set", but this
> is
> > > > minor.
> > > >
> > > > The configs for NewTopic are Map 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-14 Thread Colin McCabe
On Tue, Mar 14, 2017, at 13:36, Becket Qin wrote:
> The interface looks good overall. Thanks for the much needed work Colin.

Thanks, Becket.

> Just a few comments:
> 
> 1. I agree with Gwen that it is a little unfortunate we have to double
> the
> methods for batching interface. In the new consumer interface we only
> have
> batch interface and it seems to be working fine. Maybe we can do the same
> here by just remove all the non-batching interface.

Hmm.  The consumer seems like a different use-case, because pretty much
everyone consumes more than one record from Kafka over the lifespan of
their consumer, but a significant fraction of admin client users may
only want to delete or create one topic.

I'm not sure if there is a benefit to removing createTopic and
deleteTopic.  It seems like users who want to create or delete just one
topic would have to add more boilerplate.  I'm curious what others think
here.

> 
> 2. NewTopic.setConfigs() is a little weird, can it just be part of the
> constructor? Any specific reason to change the configs after the creation
> of a NewTopic instance?

I guess the thinking here was that the constructor contains the fields
that are mandatory (name and partition assignments / strategy), but
setting a non-default config is optional.

cheers,
Colin


> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> On Tue, Mar 14, 2017 at 10:15 AM, Ismael Juma  wrote:
> 
> > Hi Gwen,
> >
> > I agree with you that it's unfortunate that we now have 4 methods per
> > operation (including the batch ones). I personally still prefer the option
> > of using a single class to represent the parameters of each method (i.e. 1
> > batch and 1 non-batch method per operation). I would follow a very simple
> > and boring pattern for each operation:
> >
> > CreateTopicResults createTopic(CreateTopicRequest request);
> > CreateTopicsResults createTopics(CreateTopicsRequest request);
> >
> > This is simple to understand, easy to keep consistent as we add methods and
> > easy to evolve. A downside is that it's a bit more verbose than the current
> > proposal.
> >
> > Regarding LogConfig, we can't use it as it's in the core module (and in
> > Scala).
> >
> > I'm not too sure about the Unstable point. A couple of points:
> >
> > 1. A major reason why we are adding the AdminClient is so that people don't
> > have to deal with incompatible changes. So, unless we truly have to, I
> > don't think we'll be making incompatible changes after the release.
> >
> > 2. If we make changes to unstable code, it still has to go through the KIP
> > process. If they're minor changes in this release cycle, we can simply send
> > an email to the discuss thread, but if it's a major change, then it would
> > require a new round of voting (or a new KIP).
> >
> > Anyway, I think we're close on this one, but it would be good to agree on
> > the general pattern so that we can easily add more methods.
> >
> > Ismael
> >
> > On Tue, Mar 14, 2017 at 5:29 AM, Gwen Shapira  wrote:
> >
> > > I'm torn between my desire to get this in already and the fact that parts
> > > of the API feel a bit alien to Kafka.
> > >
> > > I will resolve my difficulties by giving my feedback here and then going
> > to
> > > vote +1 on the vote thread.
> > > Colin can choose whether to address my concerns now or use his "unstable"
> > > option to wait and see...
> > >
> > > My main concern is the RequestOptions objects... It was discussed
> > earlier a
> > > bit, but I'm not sure my particular concerns were addressed. I see the
> > > following issues with it:
> > > * They double the number of methods we have
> > > * They are pretty similar, so it isn't clear why we need all those
> > > different objects
> > > * All our other APIs specify timeouts either in method calls directly or
> > in
> > > the configuration for the entire object.
> > >
> > > We also typically don't use methods that start with "set", but this is
> > > minor.
> > >
> > > The configs for NewTopic are Map - shouldn't we use the
> > > LogConfig object that we already have? This will take care of
> > documentation
> > > and be similar to ProducerConfig and ConsumerConfig?
> > >
> > > My concerns aside, thank you for working on this much needed API.
> > >
> > > Gwen
> > >
> > > On Mon, Mar 13, 2017 at 3:26 PM, Colin McCabe 
> > wrote:
> > >
> > > > Hi Radai,
> > > >
> > > > Thanks for looking at the KIP again.
> > > >
> > > > On Mon, Mar 13, 2017, at 12:33, radai wrote:
> > > > > looking at the KIP as it is now, looks like all *Options objects
> > have a
> > > > > common timeout property. could it be extracted to a common
> > > > > AdminRequestOptions or something?
> > > >
> > > > Perhaps I'm missing something, but I don't think there is any reason to
> > > > extract the timeout property.  It doesn't simplify the implementation
> > > > (I've already implemented the interface in a branch, so I know this for
> > > > sure.)  It 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-14 Thread Colin McCabe
On Tue, Mar 14, 2017, at 10:15, Ismael Juma wrote:
> Hi Gwen,
> 
> I agree with you that it's unfortunate that we now have 4 methods per
> operation (including the batch ones). I personally still prefer the
> option of using a single class to represent the parameters of each method 
> (i.e.
> 1 batch and 1 non-batch method per operation).  I would follow a very simple
> and boring pattern for each operation:
> 
> CreateTopicResults createTopic(CreateTopicRequest request);
> CreateTopicsResults createTopics(CreateTopicsRequest request);
> 
> This is simple to understand, easy to keep consistent as we add methods
> and easy to evolve. A downside is that it's a bit more verbose than the
> current proposal.

It's not clear to me that request objects are simpler to understand. 
More verbose solutions tend to be less clear.  I'm curious what others
think.

One thing we talked about earlier is that there is already a set of
classes named "request" -- i.e. CreateTopicsRequest,
DeleteTopicsRequest, etc.  These classes are used inside the broker and
clients to represent RPC messages.  So if we created another set of
Request classes, it would confuse a lot of people, and lead to a "who's
on first?" situation.  "Use the request classes, not the request
classes!"  I suppose we could potentially alleviate this by renaming the
existing *Request classes to *Message classes.

> 
> Regarding LogConfig, we can't use it as it's in the core module (and in
> Scala).
> 
> I'm not too sure about the Unstable point. A couple of points:
> 
> 1. A major reason why we are adding the AdminClient is so that people
> don't have to deal with incompatible changes. So, unless we truly have to, I
> don't think we'll be making incompatible changes after the release.

Unstable APIs have worked well for streams.  A lot of the unstable APIs
they changed were more widely used by end-users than AdminClient is
likely to ever be.

In any case, this just highlights the importance of getting this in well
before the release, so that we can get feedback from real users about
what is useful, what is confusing, what could be improved, etc.

> 
> 2. If we make changes to unstable code, it still has to go through the
> KIP process. If they're minor changes in this release cycle, we can simply
> send an email to the discuss thread, but if it's a major change, then it would
> require a new round of voting (or a new KIP).
> 
> Anyway, I think we're close on this one, but it would be good to agree on
> the general pattern so that we can easily add more methods.

The problem is that a lot of this boils down to personal preference, and
it will never really be settled.  The discussion could go on forever
because everyone has their own aesthetic preferences.  For example,
there is a message later in this thread arguing that having both
createTopic and createTopics is untidy, and we should unify them into
one call.  If I agreed to that, the next message would be about how it
would be convenient to have a createTopic method, for when you were only
creating one topic :)

I think the important things in the API are:
1. supports async via CompletableFuture
2. supports batching
3. easily extensible and configurable

The other stuff really is just details that we can easily change before
the release.  For example, if you really feel strongly that request
classes are the way to go and can get everyone else to agree to that, I
will +1 it.  But let's get the basics in first and start getting useful
feedback from people building against trunk.

best,
Colin

> 
> Ismael
> 
> On Tue, Mar 14, 2017 at 5:29 AM, Gwen Shapira  wrote:
> 
> > I'm torn between my desire to get this in already and the fact that parts
> > of the API feel a bit alien to Kafka.
> >
> > I will resolve my difficulties by giving my feedback here and then going to
> > vote +1 on the vote thread.
> > Colin can choose whether to address my concerns now or use his "unstable"
> > option to wait and see...
> >
> > My main concern is the RequestOptions objects... It was discussed earlier a
> > bit, but I'm not sure my particular concerns were addressed. I see the
> > following issues with it:
> > * They double the number of methods we have
> > * They are pretty similar, so it isn't clear why we need all those
> > different objects
> > * All our other APIs specify timeouts either in method calls directly or in
> > the configuration for the entire object.
> >
> > We also typically don't use methods that start with "set", but this is
> > minor.
> >
> > The configs for NewTopic are Map - shouldn't we use the
> > LogConfig object that we already have? This will take care of documentation
> > and be similar to ProducerConfig and ConsumerConfig?
> >
> > My concerns aside, thank you for working on this much needed API.
> >
> > Gwen
> >
> > On Mon, Mar 13, 2017 at 3:26 PM, Colin McCabe  wrote:
> >
> > > Hi Radai,
> > >
> > > Thanks for looking at the KIP again.
> > >
> 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-14 Thread Colin McCabe
On Mon, Mar 13, 2017, at 22:29, Gwen Shapira wrote:
> I'm torn between my desire to get this in already and the fact that parts
> of the API feel a bit alien to Kafka.
> 
> I will resolve my difficulties by giving my feedback here and then going
> to
> vote +1 on the vote thread.
> Colin can choose whether to address my concerns now or use his "unstable"
> option to wait and see...

Thanks, Gwen.  I really appreciate all the feedback here, and the vote.
:)

> 
> My main concern is the RequestOptions objects... It was discussed earlier
> a
> bit, but I'm not sure my particular concerns were addressed. I see the
> following issues with it:
> * They double the number of methods we have
> * They are pretty similar, so it isn't clear why we need all those
> different objects
> * All our other APIs specify timeouts either in method calls directly or
> in the configuration for the entire object.

The variants without an "options" object are for the convenience of
users who want the default options.  As a practical matter, I don't see
any disadvantages to having them.  If we really feel strongly about it,
we can go back to the original proposal of having users pass "null" when
they want the defaults.

The options are different classes for each operation so that they can be
extended in the future with whatever operation-specific options we think
of.  If they were all the same type, this would not be possible (or we
would have bad things like options that are only valid for CreateTopics
getting passed to DeleteTopics, etc.)  Different types for each allow us
to have type safety here.

There is a timeout for the entire AdminClient.  This timeout is used as
the default whenever no options are specified for a call.  But one
request that end-users made over and over is the ability to set a
specific timeout for individual operations, that overrode the global
default.  They also strongly believed that the timeout should be on the
operation as a whole, not just a single part (like the time it spent
waiting for a response after sending the RPC, etc.)

> 
> We also typically don't use methods that start with "set", but this is
> minor.

Oops!  Let me fix that.  Good call.

> 
> The configs for NewTopic are Map - shouldn't we use the
> LogConfig object that we already have? This will take care of
> documentation
> and be similar to ProducerConfig and ConsumerConfig?

Like Ismael said, that's part of core, not clients...

> 
> My concerns aside, thank you for working on this much needed API.

Thanks for taking another look.

best,
Colin

> 
> Gwen
> 
> On Mon, Mar 13, 2017 at 3:26 PM, Colin McCabe  wrote:
> 
> > Hi Radai,
> >
> > Thanks for looking at the KIP again.
> >
> > On Mon, Mar 13, 2017, at 12:33, radai wrote:
> > > looking at the KIP as it is now, looks like all *Options objects have a
> > > common timeout property. could it be extracted to a common
> > > AdminRequestOptions or something?
> >
> > Perhaps I'm missing something, but I don't think there is any reason to
> > extract the timeout property.  It doesn't simplify the implementation
> > (I've already implemented the interface in a branch, so I know this for
> > sure.)  It doesn't simplify the API exposed to the users, since they
> > will still want to provide the specific option type corresponding to the
> > call.  Also, as we discussed previously in the thread (about NewTopic),
> > having lot of inheritance and base classes makes it difficult to change
> > classes over time.  It is better to simply use composition.
> >
> > I think it would be much better to get the AdminClient interface in, and
> > start iterating on it incrementally as we discover ways it could be
> > better.  This is similar to how some things in Streams were added as
> > unstable interfaces and then stabilized over time.
> >
> > best,
> > Colin
> >
> >
> > >
> > > On Thu, Mar 9, 2017 at 2:14 PM, Colin McCabe  wrote:
> > >
> > > > Hi all,
> > > >
> > > > We've been discussing this for a while (about a month) and I think
> > > > people have made some great points that improved the proposal.  In
> > > > particular, adding async and batching was important.  I've also been
> > > > talking with some end-users who would like to make use of this API.
> > > > Once this is in, we can iterate on it before the release, and it will
> > > > also unblock a lot of other admin proposals.  I think it would be good
> > > > to start the vote in a little bit, assuming there are no objections.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Wed, Mar 8, 2017, at 17:09, Colin McCabe wrote:
> > > > > On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> > > > > > Thanks Colin.
> > > > > >
> > > > > > I am familiar with the protocol semantics, but we need to document
> > the
> > > > > > API
> > > > > > for users who don't know the protocol. I still think it would be
> > > > valuable
> > > > > > to have some examples of how the API would be used 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-14 Thread Becket Qin
The interface looks good overall. Thanks for the much needed work Colin.
Just a few comments:

1. I agree with Gwen that it is a little unfortunate we have to double the
methods for batching interface. In the new consumer interface we only have
batch interface and it seems to be working fine. Maybe we can do the same
here by just remove all the non-batching interface.

2. NewTopic.setConfigs() is a little weird, can it just be part of the
constructor? Any specific reason to change the configs after the creation
of a NewTopic instance?

Thanks,

Jiangjie (Becket) Qin

On Tue, Mar 14, 2017 at 10:15 AM, Ismael Juma  wrote:

> Hi Gwen,
>
> I agree with you that it's unfortunate that we now have 4 methods per
> operation (including the batch ones). I personally still prefer the option
> of using a single class to represent the parameters of each method (i.e. 1
> batch and 1 non-batch method per operation). I would follow a very simple
> and boring pattern for each operation:
>
> CreateTopicResults createTopic(CreateTopicRequest request);
> CreateTopicsResults createTopics(CreateTopicsRequest request);
>
> This is simple to understand, easy to keep consistent as we add methods and
> easy to evolve. A downside is that it's a bit more verbose than the current
> proposal.
>
> Regarding LogConfig, we can't use it as it's in the core module (and in
> Scala).
>
> I'm not too sure about the Unstable point. A couple of points:
>
> 1. A major reason why we are adding the AdminClient is so that people don't
> have to deal with incompatible changes. So, unless we truly have to, I
> don't think we'll be making incompatible changes after the release.
>
> 2. If we make changes to unstable code, it still has to go through the KIP
> process. If they're minor changes in this release cycle, we can simply send
> an email to the discuss thread, but if it's a major change, then it would
> require a new round of voting (or a new KIP).
>
> Anyway, I think we're close on this one, but it would be good to agree on
> the general pattern so that we can easily add more methods.
>
> Ismael
>
> On Tue, Mar 14, 2017 at 5:29 AM, Gwen Shapira  wrote:
>
> > I'm torn between my desire to get this in already and the fact that parts
> > of the API feel a bit alien to Kafka.
> >
> > I will resolve my difficulties by giving my feedback here and then going
> to
> > vote +1 on the vote thread.
> > Colin can choose whether to address my concerns now or use his "unstable"
> > option to wait and see...
> >
> > My main concern is the RequestOptions objects... It was discussed
> earlier a
> > bit, but I'm not sure my particular concerns were addressed. I see the
> > following issues with it:
> > * They double the number of methods we have
> > * They are pretty similar, so it isn't clear why we need all those
> > different objects
> > * All our other APIs specify timeouts either in method calls directly or
> in
> > the configuration for the entire object.
> >
> > We also typically don't use methods that start with "set", but this is
> > minor.
> >
> > The configs for NewTopic are Map - shouldn't we use the
> > LogConfig object that we already have? This will take care of
> documentation
> > and be similar to ProducerConfig and ConsumerConfig?
> >
> > My concerns aside, thank you for working on this much needed API.
> >
> > Gwen
> >
> > On Mon, Mar 13, 2017 at 3:26 PM, Colin McCabe 
> wrote:
> >
> > > Hi Radai,
> > >
> > > Thanks for looking at the KIP again.
> > >
> > > On Mon, Mar 13, 2017, at 12:33, radai wrote:
> > > > looking at the KIP as it is now, looks like all *Options objects
> have a
> > > > common timeout property. could it be extracted to a common
> > > > AdminRequestOptions or something?
> > >
> > > Perhaps I'm missing something, but I don't think there is any reason to
> > > extract the timeout property.  It doesn't simplify the implementation
> > > (I've already implemented the interface in a branch, so I know this for
> > > sure.)  It doesn't simplify the API exposed to the users, since they
> > > will still want to provide the specific option type corresponding to
> the
> > > call.  Also, as we discussed previously in the thread (about NewTopic),
> > > having lot of inheritance and base classes makes it difficult to change
> > > classes over time.  It is better to simply use composition.
> > >
> > > I think it would be much better to get the AdminClient interface in,
> and
> > > start iterating on it incrementally as we discover ways it could be
> > > better.  This is similar to how some things in Streams were added as
> > > unstable interfaces and then stabilized over time.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > >
> > > > On Thu, Mar 9, 2017 at 2:14 PM, Colin McCabe 
> > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > We've been discussing this for a while (about a month) and I think
> > > > > people have made some great points that 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-14 Thread Ismael Juma
Hi Gwen,

I agree with you that it's unfortunate that we now have 4 methods per
operation (including the batch ones). I personally still prefer the option
of using a single class to represent the parameters of each method (i.e. 1
batch and 1 non-batch method per operation). I would follow a very simple
and boring pattern for each operation:

CreateTopicResults createTopic(CreateTopicRequest request);
CreateTopicsResults createTopics(CreateTopicsRequest request);

This is simple to understand, easy to keep consistent as we add methods and
easy to evolve. A downside is that it's a bit more verbose than the current
proposal.

Regarding LogConfig, we can't use it as it's in the core module (and in
Scala).

I'm not too sure about the Unstable point. A couple of points:

1. A major reason why we are adding the AdminClient is so that people don't
have to deal with incompatible changes. So, unless we truly have to, I
don't think we'll be making incompatible changes after the release.

2. If we make changes to unstable code, it still has to go through the KIP
process. If they're minor changes in this release cycle, we can simply send
an email to the discuss thread, but if it's a major change, then it would
require a new round of voting (or a new KIP).

Anyway, I think we're close on this one, but it would be good to agree on
the general pattern so that we can easily add more methods.

Ismael

On Tue, Mar 14, 2017 at 5:29 AM, Gwen Shapira  wrote:

> I'm torn between my desire to get this in already and the fact that parts
> of the API feel a bit alien to Kafka.
>
> I will resolve my difficulties by giving my feedback here and then going to
> vote +1 on the vote thread.
> Colin can choose whether to address my concerns now or use his "unstable"
> option to wait and see...
>
> My main concern is the RequestOptions objects... It was discussed earlier a
> bit, but I'm not sure my particular concerns were addressed. I see the
> following issues with it:
> * They double the number of methods we have
> * They are pretty similar, so it isn't clear why we need all those
> different objects
> * All our other APIs specify timeouts either in method calls directly or in
> the configuration for the entire object.
>
> We also typically don't use methods that start with "set", but this is
> minor.
>
> The configs for NewTopic are Map - shouldn't we use the
> LogConfig object that we already have? This will take care of documentation
> and be similar to ProducerConfig and ConsumerConfig?
>
> My concerns aside, thank you for working on this much needed API.
>
> Gwen
>
> On Mon, Mar 13, 2017 at 3:26 PM, Colin McCabe  wrote:
>
> > Hi Radai,
> >
> > Thanks for looking at the KIP again.
> >
> > On Mon, Mar 13, 2017, at 12:33, radai wrote:
> > > looking at the KIP as it is now, looks like all *Options objects have a
> > > common timeout property. could it be extracted to a common
> > > AdminRequestOptions or something?
> >
> > Perhaps I'm missing something, but I don't think there is any reason to
> > extract the timeout property.  It doesn't simplify the implementation
> > (I've already implemented the interface in a branch, so I know this for
> > sure.)  It doesn't simplify the API exposed to the users, since they
> > will still want to provide the specific option type corresponding to the
> > call.  Also, as we discussed previously in the thread (about NewTopic),
> > having lot of inheritance and base classes makes it difficult to change
> > classes over time.  It is better to simply use composition.
> >
> > I think it would be much better to get the AdminClient interface in, and
> > start iterating on it incrementally as we discover ways it could be
> > better.  This is similar to how some things in Streams were added as
> > unstable interfaces and then stabilized over time.
> >
> > best,
> > Colin
> >
> >
> > >
> > > On Thu, Mar 9, 2017 at 2:14 PM, Colin McCabe 
> wrote:
> > >
> > > > Hi all,
> > > >
> > > > We've been discussing this for a while (about a month) and I think
> > > > people have made some great points that improved the proposal.  In
> > > > particular, adding async and batching was important.  I've also been
> > > > talking with some end-users who would like to make use of this API.
> > > > Once this is in, we can iterate on it before the release, and it will
> > > > also unblock a lot of other admin proposals.  I think it would be
> good
> > > > to start the vote in a little bit, assuming there are no objections.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Wed, Mar 8, 2017, at 17:09, Colin McCabe wrote:
> > > > > On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> > > > > > Thanks Colin.
> > > > > >
> > > > > > I am familiar with the protocol semantics, but we need to
> document
> > the
> > > > > > API
> > > > > > for users who don't know the protocol. I still think it would be
> > > > valuable
> > > > > > to have some 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-13 Thread Gwen Shapira
I'm torn between my desire to get this in already and the fact that parts
of the API feel a bit alien to Kafka.

I will resolve my difficulties by giving my feedback here and then going to
vote +1 on the vote thread.
Colin can choose whether to address my concerns now or use his "unstable"
option to wait and see...

My main concern is the RequestOptions objects... It was discussed earlier a
bit, but I'm not sure my particular concerns were addressed. I see the
following issues with it:
* They double the number of methods we have
* They are pretty similar, so it isn't clear why we need all those
different objects
* All our other APIs specify timeouts either in method calls directly or in
the configuration for the entire object.

We also typically don't use methods that start with "set", but this is
minor.

The configs for NewTopic are Map - shouldn't we use the
LogConfig object that we already have? This will take care of documentation
and be similar to ProducerConfig and ConsumerConfig?

My concerns aside, thank you for working on this much needed API.

Gwen

On Mon, Mar 13, 2017 at 3:26 PM, Colin McCabe  wrote:

> Hi Radai,
>
> Thanks for looking at the KIP again.
>
> On Mon, Mar 13, 2017, at 12:33, radai wrote:
> > looking at the KIP as it is now, looks like all *Options objects have a
> > common timeout property. could it be extracted to a common
> > AdminRequestOptions or something?
>
> Perhaps I'm missing something, but I don't think there is any reason to
> extract the timeout property.  It doesn't simplify the implementation
> (I've already implemented the interface in a branch, so I know this for
> sure.)  It doesn't simplify the API exposed to the users, since they
> will still want to provide the specific option type corresponding to the
> call.  Also, as we discussed previously in the thread (about NewTopic),
> having lot of inheritance and base classes makes it difficult to change
> classes over time.  It is better to simply use composition.
>
> I think it would be much better to get the AdminClient interface in, and
> start iterating on it incrementally as we discover ways it could be
> better.  This is similar to how some things in Streams were added as
> unstable interfaces and then stabilized over time.
>
> best,
> Colin
>
>
> >
> > On Thu, Mar 9, 2017 at 2:14 PM, Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > We've been discussing this for a while (about a month) and I think
> > > people have made some great points that improved the proposal.  In
> > > particular, adding async and batching was important.  I've also been
> > > talking with some end-users who would like to make use of this API.
> > > Once this is in, we can iterate on it before the release, and it will
> > > also unblock a lot of other admin proposals.  I think it would be good
> > > to start the vote in a little bit, assuming there are no objections.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Wed, Mar 8, 2017, at 17:09, Colin McCabe wrote:
> > > > On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> > > > > Thanks Colin.
> > > > >
> > > > > I am familiar with the protocol semantics, but we need to document
> the
> > > > > API
> > > > > for users who don't know the protocol. I still think it would be
> > > valuable
> > > > > to have some examples of how the API would be used for common use
> > > cases.
> > > >
> > > > Getting the version of all nodes in the cluster:
> > > >   Map nodesToVersions =
> > > > adminClient.listNodes().nodes().thenApply(
> > > >   nodes -> adminClient.apiVersions(nodes)).all().get();
> > > >
> > > > Creating a topic:
> > > >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short)
> > > >   3)).all().get();
> > > >
> > > > Validating that a topic can be created (but not creating it):
> > > >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short) 3),
> > > > new CreateTopicOptions().setValidateOnly(true)).all().get();
> > > >
> > > > > For example, say someone creates a topic and then produces to it.
> What
> > > > > would be the recommended way to do that?
> > > >
> > > > Once the future returned by createTopics has successfully completed,
> it
> > > > should be possible to produce to the topic.
> > > >
> > > > There are a few warts that are definitely worth calling out.  These
> are
> > > > things that need to be fixed at the protocol layer, so they're
> outside
> > > > the scope of this KIP.  But you made a good point that we need to
> > > > document this well.  Here's my list (I wonder if anyone has more?):
> > > >
> > > > * If auto.create.topics.enable is true on the brokers,
> > > > AdminClient#describeTopic(topicName) may create a topic named
> topicName.
> > > >  There are two workarounds: either use AdminClient#listTopics and
> ensure
> > > > that the topic is present before describing, or disable
> > > > auto.create.topics.enable.
> > > >
> > > > * If delete.topic.enable 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-13 Thread Colin McCabe
Hi Radai,

Thanks for looking at the KIP again.

On Mon, Mar 13, 2017, at 12:33, radai wrote:
> looking at the KIP as it is now, looks like all *Options objects have a
> common timeout property. could it be extracted to a common
> AdminRequestOptions or something?

Perhaps I'm missing something, but I don't think there is any reason to
extract the timeout property.  It doesn't simplify the implementation
(I've already implemented the interface in a branch, so I know this for
sure.)  It doesn't simplify the API exposed to the users, since they
will still want to provide the specific option type corresponding to the
call.  Also, as we discussed previously in the thread (about NewTopic),
having lot of inheritance and base classes makes it difficult to change
classes over time.  It is better to simply use composition.

I think it would be much better to get the AdminClient interface in, and
start iterating on it incrementally as we discover ways it could be
better.  This is similar to how some things in Streams were added as
unstable interfaces and then stabilized over time.

best,
Colin


> 
> On Thu, Mar 9, 2017 at 2:14 PM, Colin McCabe  wrote:
> 
> > Hi all,
> >
> > We've been discussing this for a while (about a month) and I think
> > people have made some great points that improved the proposal.  In
> > particular, adding async and batching was important.  I've also been
> > talking with some end-users who would like to make use of this API.
> > Once this is in, we can iterate on it before the release, and it will
> > also unblock a lot of other admin proposals.  I think it would be good
> > to start the vote in a little bit, assuming there are no objections.
> >
> > best,
> > Colin
> >
> >
> > On Wed, Mar 8, 2017, at 17:09, Colin McCabe wrote:
> > > On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> > > > Thanks Colin.
> > > >
> > > > I am familiar with the protocol semantics, but we need to document the
> > > > API
> > > > for users who don't know the protocol. I still think it would be
> > valuable
> > > > to have some examples of how the API would be used for common use
> > cases.
> > >
> > > Getting the version of all nodes in the cluster:
> > >   Map nodesToVersions =
> > > adminClient.listNodes().nodes().thenApply(
> > >   nodes -> adminClient.apiVersions(nodes)).all().get();
> > >
> > > Creating a topic:
> > >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short)
> > >   3)).all().get();
> > >
> > > Validating that a topic can be created (but not creating it):
> > >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short) 3),
> > > new CreateTopicOptions().setValidateOnly(true)).all().get();
> > >
> > > > For example, say someone creates a topic and then produces to it. What
> > > > would be the recommended way to do that?
> > >
> > > Once the future returned by createTopics has successfully completed, it
> > > should be possible to produce to the topic.
> > >
> > > There are a few warts that are definitely worth calling out.  These are
> > > things that need to be fixed at the protocol layer, so they're outside
> > > the scope of this KIP.  But you made a good point that we need to
> > > document this well.  Here's my list (I wonder if anyone has more?):
> > >
> > > * If auto.create.topics.enable is true on the brokers,
> > > AdminClient#describeTopic(topicName) may create a topic named topicName.
> > >  There are two workarounds: either use AdminClient#listTopics and ensure
> > > that the topic is present before describing, or disable
> > > auto.create.topics.enable.
> > >
> > > * If delete.topic.enable is false on the brokers,
> > > AdminClient#deleteTopic(topicName) will mark topicName for deletion, but
> > > not actually delete it.  deleteTopic will return success in this case.
> > >
> > > * It may take several seconds after AdminClient#deleteTopic returns
> > > success for all the brokers to become aware that the topic is gone.
> > > During this time, AdminClient#listTopics and AdminClient#describeTopic
> > > may continue to return information about the deleted topic.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > >
> > > > Ismael
> > > >
> > > > On Mon, Mar 6, 2017 at 7:43 PM, Colin McCabe 
> > wrote:
> > > >
> > > > > On Mon, Mar 6, 2017, at 05:50, Ismael Juma wrote:
> > > > > > Thanks Colin. It seems like you replied to me accidentally instead
> > of the
> > > > > > list, so leaving your reply below for the benefit of others.
> > > > >
> > > > > Thanks, Ismael.  I actually realized my mistake right after I sent to
> > > > > you, and re-posted it to the mailing list instead of sending
> > directly.
> > > > > Sigh...
> > > > >
> > > > > >
> > > > > > Regarding the disadvantage of having to hunt through the request
> > class,
> > > > > > don't people have to do that anyway with the Options classes?
> > > > >
> > > > > A lot of people will simply choose the default options, until they
> > have
> > > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-13 Thread radai
looking at the KIP as it is now, looks like all *Options objects have a
common timeout property. could it be extracted to a common
AdminRequestOptions or something?

On Thu, Mar 9, 2017 at 2:14 PM, Colin McCabe  wrote:

> Hi all,
>
> We've been discussing this for a while (about a month) and I think
> people have made some great points that improved the proposal.  In
> particular, adding async and batching was important.  I've also been
> talking with some end-users who would like to make use of this API.
> Once this is in, we can iterate on it before the release, and it will
> also unblock a lot of other admin proposals.  I think it would be good
> to start the vote in a little bit, assuming there are no objections.
>
> best,
> Colin
>
>
> On Wed, Mar 8, 2017, at 17:09, Colin McCabe wrote:
> > On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> > > Thanks Colin.
> > >
> > > I am familiar with the protocol semantics, but we need to document the
> > > API
> > > for users who don't know the protocol. I still think it would be
> valuable
> > > to have some examples of how the API would be used for common use
> cases.
> >
> > Getting the version of all nodes in the cluster:
> >   Map nodesToVersions =
> > adminClient.listNodes().nodes().thenApply(
> >   nodes -> adminClient.apiVersions(nodes)).all().get();
> >
> > Creating a topic:
> >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short)
> >   3)).all().get();
> >
> > Validating that a topic can be created (but not creating it):
> >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short) 3),
> > new CreateTopicOptions().setValidateOnly(true)).all().get();
> >
> > > For example, say someone creates a topic and then produces to it. What
> > > would be the recommended way to do that?
> >
> > Once the future returned by createTopics has successfully completed, it
> > should be possible to produce to the topic.
> >
> > There are a few warts that are definitely worth calling out.  These are
> > things that need to be fixed at the protocol layer, so they're outside
> > the scope of this KIP.  But you made a good point that we need to
> > document this well.  Here's my list (I wonder if anyone has more?):
> >
> > * If auto.create.topics.enable is true on the brokers,
> > AdminClient#describeTopic(topicName) may create a topic named topicName.
> >  There are two workarounds: either use AdminClient#listTopics and ensure
> > that the topic is present before describing, or disable
> > auto.create.topics.enable.
> >
> > * If delete.topic.enable is false on the brokers,
> > AdminClient#deleteTopic(topicName) will mark topicName for deletion, but
> > not actually delete it.  deleteTopic will return success in this case.
> >
> > * It may take several seconds after AdminClient#deleteTopic returns
> > success for all the brokers to become aware that the topic is gone.
> > During this time, AdminClient#listTopics and AdminClient#describeTopic
> > may continue to return information about the deleted topic.
> >
> > best,
> > Colin
> >
> >
> > >
> > > Ismael
> > >
> > > On Mon, Mar 6, 2017 at 7:43 PM, Colin McCabe 
> wrote:
> > >
> > > > On Mon, Mar 6, 2017, at 05:50, Ismael Juma wrote:
> > > > > Thanks Colin. It seems like you replied to me accidentally instead
> of the
> > > > > list, so leaving your reply below for the benefit of others.
> > > >
> > > > Thanks, Ismael.  I actually realized my mistake right after I sent to
> > > > you, and re-posted it to the mailing list instead of sending
> directly.
> > > > Sigh...
> > > >
> > > > >
> > > > > Regarding the disadvantage of having to hunt through the request
> class,
> > > > > don't people have to do that anyway with the Options classes?
> > > >
> > > > A lot of people will simply choose the default options, until they
> have
> > > > a reason to do otherwise (for example, they want a longer or shorter
> > > > timeout, etc.)
> > > >
> > > > >
> > > > > Aside from that, it would be great if the KIP included more
> detailed
> > > > > javadoc for each method including information about potential
> exceptions.
> > > >
> > > > That's a good question.  Because this is an asynchronous API, methods
> > > > never throw exceptions.  Instead, if you call get() / whenComplete()
> /
> > > > isCompletedExceptionally() / etc. on one of the CompletableFuture
> > > > objects, you will get the exception.  This is to allow Node.js-style
> > > > completion chaining.  I will add this explanation to the KIP.
> > > >
> > > > > I'm particularly interested in what a user can expect if a create
> topics
> > > > > succeeds versus what they can expect if a timeout exception is
> thrown. It
> > > > > would be good to consider partial failures as well.
> > > >
> > > > This is spelled out by KIP-4.
> > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+
> > > > Command+line+and+centralized+administrative+operations
> > > >
> > > > Specifically,
> > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-09 Thread Colin McCabe
Hi all,

We've been discussing this for a while (about a month) and I think
people have made some great points that improved the proposal.  In
particular, adding async and batching was important.  I've also been
talking with some end-users who would like to make use of this API. 
Once this is in, we can iterate on it before the release, and it will
also unblock a lot of other admin proposals.  I think it would be good
to start the vote in a little bit, assuming there are no objections.

best,
Colin


On Wed, Mar 8, 2017, at 17:09, Colin McCabe wrote:
> On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> > Thanks Colin.
> > 
> > I am familiar with the protocol semantics, but we need to document the
> > API
> > for users who don't know the protocol. I still think it would be valuable
> > to have some examples of how the API would be used for common use cases.
> 
> Getting the version of all nodes in the cluster:
>   Map nodesToVersions =
> adminClient.listNodes().nodes().thenApply(
>   nodes -> adminClient.apiVersions(nodes)).all().get();
> 
> Creating a topic:
>   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short)
>   3)).all().get();
> 
> Validating that a topic can be created (but not creating it):
>   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short) 3),
> new CreateTopicOptions().setValidateOnly(true)).all().get();
> 
> > For example, say someone creates a topic and then produces to it. What
> > would be the recommended way to do that?
> 
> Once the future returned by createTopics has successfully completed, it
> should be possible to produce to the topic.
> 
> There are a few warts that are definitely worth calling out.  These are
> things that need to be fixed at the protocol layer, so they're outside
> the scope of this KIP.  But you made a good point that we need to
> document this well.  Here's my list (I wonder if anyone has more?):
> 
> * If auto.create.topics.enable is true on the brokers,
> AdminClient#describeTopic(topicName) may create a topic named topicName.
>  There are two workarounds: either use AdminClient#listTopics and ensure
> that the topic is present before describing, or disable
> auto.create.topics.enable.
> 
> * If delete.topic.enable is false on the brokers,
> AdminClient#deleteTopic(topicName) will mark topicName for deletion, but
> not actually delete it.  deleteTopic will return success in this case.
> 
> * It may take several seconds after AdminClient#deleteTopic returns
> success for all the brokers to become aware that the topic is gone. 
> During this time, AdminClient#listTopics and AdminClient#describeTopic
> may continue to return information about the deleted topic.
> 
> best,
> Colin
> 
> 
> > 
> > Ismael
> > 
> > On Mon, Mar 6, 2017 at 7:43 PM, Colin McCabe  wrote:
> > 
> > > On Mon, Mar 6, 2017, at 05:50, Ismael Juma wrote:
> > > > Thanks Colin. It seems like you replied to me accidentally instead of 
> > > > the
> > > > list, so leaving your reply below for the benefit of others.
> > >
> > > Thanks, Ismael.  I actually realized my mistake right after I sent to
> > > you, and re-posted it to the mailing list instead of sending directly.
> > > Sigh...
> > >
> > > >
> > > > Regarding the disadvantage of having to hunt through the request class,
> > > > don't people have to do that anyway with the Options classes?
> > >
> > > A lot of people will simply choose the default options, until they have
> > > a reason to do otherwise (for example, they want a longer or shorter
> > > timeout, etc.)
> > >
> > > >
> > > > Aside from that, it would be great if the KIP included more detailed
> > > > javadoc for each method including information about potential 
> > > > exceptions.
> > >
> > > That's a good question.  Because this is an asynchronous API, methods
> > > never throw exceptions.  Instead, if you call get() / whenComplete() /
> > > isCompletedExceptionally() / etc. on one of the CompletableFuture
> > > objects, you will get the exception.  This is to allow Node.js-style
> > > completion chaining.  I will add this explanation to the KIP.
> > >
> > > > I'm particularly interested in what a user can expect if a create topics
> > > > succeeds versus what they can expect if a timeout exception is thrown. 
> > > > It
> > > > would be good to consider partial failures as well.
> > >
> > > This is spelled out by KIP-4.
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+
> > > Command+line+and+centralized+administrative+operations
> > >
> > > Specifically,
> > >
> > > > If a timeout error occurs [in CreateTopic], the topic could still be
> > > > created successfully at a later time. Its up to the client to query
> > > > for the state at that point.
> > >
> > > Since we're specifically not changing the server as part of this KIP,
> > > those semantics will still be in force.  Of course, there are plenty of
> > > other exceptions that you can get from CreateTopics that are more
> > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-08 Thread Colin McCabe
On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> Thanks Colin.
> 
> I am familiar with the protocol semantics, but we need to document the
> API
> for users who don't know the protocol. I still think it would be valuable
> to have some examples of how the API would be used for common use cases.

Getting the version of all nodes in the cluster:
  Map nodesToVersions =
adminClient.listNodes().nodes().thenApply(
  nodes -> adminClient.apiVersions(nodes)).all().get();

Creating a topic:
  adminClient.createTopic(new NewTopic("myNewTopic", 3, (short)
  3)).all().get();

Validating that a topic can be created (but not creating it):
  adminClient.createTopic(new NewTopic("myNewTopic", 3, (short) 3),
new CreateTopicOptions().setValidateOnly(true)).all().get();

> For example, say someone creates a topic and then produces to it. What
> would be the recommended way to do that?

Once the future returned by createTopics has successfully completed, it
should be possible to produce to the topic.

There are a few warts that are definitely worth calling out.  These are
things that need to be fixed at the protocol layer, so they're outside
the scope of this KIP.  But you made a good point that we need to
document this well.  Here's my list (I wonder if anyone has more?):

* If auto.create.topics.enable is true on the brokers,
AdminClient#describeTopic(topicName) may create a topic named topicName.
 There are two workarounds: either use AdminClient#listTopics and ensure
that the topic is present before describing, or disable
auto.create.topics.enable.

* If delete.topic.enable is false on the brokers,
AdminClient#deleteTopic(topicName) will mark topicName for deletion, but
not actually delete it.  deleteTopic will return success in this case.

* It may take several seconds after AdminClient#deleteTopic returns
success for all the brokers to become aware that the topic is gone. 
During this time, AdminClient#listTopics and AdminClient#describeTopic
may continue to return information about the deleted topic.

best,
Colin


> 
> Ismael
> 
> On Mon, Mar 6, 2017 at 7:43 PM, Colin McCabe  wrote:
> 
> > On Mon, Mar 6, 2017, at 05:50, Ismael Juma wrote:
> > > Thanks Colin. It seems like you replied to me accidentally instead of the
> > > list, so leaving your reply below for the benefit of others.
> >
> > Thanks, Ismael.  I actually realized my mistake right after I sent to
> > you, and re-posted it to the mailing list instead of sending directly.
> > Sigh...
> >
> > >
> > > Regarding the disadvantage of having to hunt through the request class,
> > > don't people have to do that anyway with the Options classes?
> >
> > A lot of people will simply choose the default options, until they have
> > a reason to do otherwise (for example, they want a longer or shorter
> > timeout, etc.)
> >
> > >
> > > Aside from that, it would be great if the KIP included more detailed
> > > javadoc for each method including information about potential exceptions.
> >
> > That's a good question.  Because this is an asynchronous API, methods
> > never throw exceptions.  Instead, if you call get() / whenComplete() /
> > isCompletedExceptionally() / etc. on one of the CompletableFuture
> > objects, you will get the exception.  This is to allow Node.js-style
> > completion chaining.  I will add this explanation to the KIP.
> >
> > > I'm particularly interested in what a user can expect if a create topics
> > > succeeds versus what they can expect if a timeout exception is thrown. It
> > > would be good to consider partial failures as well.
> >
> > This is spelled out by KIP-4.
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+
> > Command+line+and+centralized+administrative+operations
> >
> > Specifically,
> >
> > > If a timeout error occurs [in CreateTopic], the topic could still be
> > > created successfully at a later time. Its up to the client to query
> > > for the state at that point.
> >
> > Since we're specifically not changing the server as part of this KIP,
> > those semantics will still be in force.  Of course, there are plenty of
> > other exceptions that you can get from CreateTopics that are more
> > meaningful, such as permission-related or network-related ones.  But if
> > you get a timeout, the operation may or may not have succeeded.
> >
> > Could we fix the timeout problem?  Sort of.  We could implement
> > something like a retry cache.  The brokers would have to maintain a
> > cache of operations (and their results) which had succeeded or failed.
> > Then, if an RPC got interrupted after the server had performed it, but
> > before the client had received the response message, the client could
> > simply reconnect on another TCP session and ask the broker for the
> > result of the previous operation.  The broker could look up the result
> > in the cache and re-send it.
> >
> > This fix works, but it is very complex.  The cache requires space in
> > memory (and to 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-06 Thread Ismael Juma
Thanks Colin.

I am familiar with the protocol semantics, but we need to document the API
for users who don't know the protocol. I still think it would be valuable
to have some examples of how the API would be used for common use cases.
For example, say someone creates a topic and then produces to it. What
would be the recommended way to do that?

Ismael

On Mon, Mar 6, 2017 at 7:43 PM, Colin McCabe  wrote:

> On Mon, Mar 6, 2017, at 05:50, Ismael Juma wrote:
> > Thanks Colin. It seems like you replied to me accidentally instead of the
> > list, so leaving your reply below for the benefit of others.
>
> Thanks, Ismael.  I actually realized my mistake right after I sent to
> you, and re-posted it to the mailing list instead of sending directly.
> Sigh...
>
> >
> > Regarding the disadvantage of having to hunt through the request class,
> > don't people have to do that anyway with the Options classes?
>
> A lot of people will simply choose the default options, until they have
> a reason to do otherwise (for example, they want a longer or shorter
> timeout, etc.)
>
> >
> > Aside from that, it would be great if the KIP included more detailed
> > javadoc for each method including information about potential exceptions.
>
> That's a good question.  Because this is an asynchronous API, methods
> never throw exceptions.  Instead, if you call get() / whenComplete() /
> isCompletedExceptionally() / etc. on one of the CompletableFuture
> objects, you will get the exception.  This is to allow Node.js-style
> completion chaining.  I will add this explanation to the KIP.
>
> > I'm particularly interested in what a user can expect if a create topics
> > succeeds versus what they can expect if a timeout exception is thrown. It
> > would be good to consider partial failures as well.
>
> This is spelled out by KIP-4.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+
> Command+line+and+centralized+administrative+operations
>
> Specifically,
>
> > If a timeout error occurs [in CreateTopic], the topic could still be
> > created successfully at a later time. Its up to the client to query
> > for the state at that point.
>
> Since we're specifically not changing the server as part of this KIP,
> those semantics will still be in force.  Of course, there are plenty of
> other exceptions that you can get from CreateTopics that are more
> meaningful, such as permission-related or network-related ones.  But if
> you get a timeout, the operation may or may not have succeeded.
>
> Could we fix the timeout problem?  Sort of.  We could implement
> something like a retry cache.  The brokers would have to maintain a
> cache of operations (and their results) which had succeeded or failed.
> Then, if an RPC got interrupted after the server had performed it, but
> before the client had received the response message, the client could
> simply reconnect on another TCP session and ask the broker for the
> result of the previous operation.  The broker could look up the result
> in the cache and re-send it.
>
> This fix works, but it is very complex.  The cache requires space in
> memory (and to do it perfectly, you also want to persist the cache to
> disk in case the broker restarts and the client re-appears).  The fix
> also requires the client to wait for an indefinite amount of time for
> the server to come back.  If the client ever "gives up" and just throws
> a timeout exception, we are back to not knowing what happened on the
> server.
>
> In any case, I think we should discuss RPC change in a separate KIP...
> the scope is already big enough here.  Also, in practice, users have
> workarounds for cases where there are timeouts or failures to
> communicate.
>
> best,
> Colin
>
> >
> > Ismael
> >
> > On Fri, Mar 3, 2017 at 9:37 PM, Colin McCabe  wrote:
> >
> > > On Fri, Mar 3, 2017, at 06:41, Ismael Juma wrote:
> > > > Hi Colin,
> > > >
> > > > I still need to do a detailed review, but I have a couple of
> > > > comments/questions:
> > > >
> > > > 1. I am not sure about having the options/response classes as inner
> > > > classes
> > > > of the interface. It means that file containing the interface will be
> > > > huge
> > > > eventually. And the classes are not necessarily related either. Why
> not
> > > > use
> > > > a separate package for them?
> > >
> > > Yeah, I think it's reasonable to make these top-level classes and put
> > > them in separate files.  We can put them all in
> > > org.apache.kafka.clients.admin.
> > >
> > > >
> > > > 2. Can you elaborate on how one decides one goes in the Options class
> > > > versus the first parameter?
> > >
> > > I guess I think of options as things that you don't have to set.  For
> > > example, when deleting a topic, you must supply the topic name, but
> > > supplying a non-default timeout is optional.
> > >
> > > > I wonder if it would be simpler to just have a
> > > > single parameter. In that case it should probably be called a
> Request as
> 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-06 Thread Colin McCabe
On Mon, Mar 6, 2017, at 05:50, Ismael Juma wrote:
> Thanks Colin. It seems like you replied to me accidentally instead of the
> list, so leaving your reply below for the benefit of others.

Thanks, Ismael.  I actually realized my mistake right after I sent to
you, and re-posted it to the mailing list instead of sending directly. 
Sigh...

> 
> Regarding the disadvantage of having to hunt through the request class,
> don't people have to do that anyway with the Options classes?

A lot of people will simply choose the default options, until they have
a reason to do otherwise (for example, they want a longer or shorter
timeout, etc.)

> 
> Aside from that, it would be great if the KIP included more detailed
> javadoc for each method including information about potential exceptions.

That's a good question.  Because this is an asynchronous API, methods
never throw exceptions.  Instead, if you call get() / whenComplete() /
isCompletedExceptionally() / etc. on one of the CompletableFuture
objects, you will get the exception.  This is to allow Node.js-style
completion chaining.  I will add this explanation to the KIP.

> I'm particularly interested in what a user can expect if a create topics
> succeeds versus what they can expect if a timeout exception is thrown. It
> would be good to consider partial failures as well.

This is spelled out by KIP-4.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations

Specifically,

> If a timeout error occurs [in CreateTopic], the topic could still be
> created successfully at a later time. Its up to the client to query
> for the state at that point.

Since we're specifically not changing the server as part of this KIP,
those semantics will still be in force.  Of course, there are plenty of
other exceptions that you can get from CreateTopics that are more
meaningful, such as permission-related or network-related ones.  But if
you get a timeout, the operation may or may not have succeeded.

Could we fix the timeout problem?  Sort of.  We could implement
something like a retry cache.  The brokers would have to maintain a
cache of operations (and their results) which had succeeded or failed. 
Then, if an RPC got interrupted after the server had performed it, but
before the client had received the response message, the client could
simply reconnect on another TCP session and ask the broker for the
result of the previous operation.  The broker could look up the result
in the cache and re-send it.

This fix works, but it is very complex.  The cache requires space in
memory (and to do it perfectly, you also want to persist the cache to
disk in case the broker restarts and the client re-appears).  The fix
also requires the client to wait for an indefinite amount of time for
the server to come back.  If the client ever "gives up" and just throws
a timeout exception, we are back to not knowing what happened on the
server.

In any case, I think we should discuss RPC change in a separate KIP...
the scope is already big enough here.  Also, in practice, users have
workarounds for cases where there are timeouts or failures to
communicate.

best,
Colin

> 
> Ismael
> 
> On Fri, Mar 3, 2017 at 9:37 PM, Colin McCabe  wrote:
> 
> > On Fri, Mar 3, 2017, at 06:41, Ismael Juma wrote:
> > > Hi Colin,
> > >
> > > I still need to do a detailed review, but I have a couple of
> > > comments/questions:
> > >
> > > 1. I am not sure about having the options/response classes as inner
> > > classes
> > > of the interface. It means that file containing the interface will be
> > > huge
> > > eventually. And the classes are not necessarily related either. Why not
> > > use
> > > a separate package for them?
> >
> > Yeah, I think it's reasonable to make these top-level classes and put
> > them in separate files.  We can put them all in
> > org.apache.kafka.clients.admin.
> >
> > >
> > > 2. Can you elaborate on how one decides one goes in the Options class
> > > versus the first parameter?
> >
> > I guess I think of options as things that you don't have to set.  For
> > example, when deleting a topic, you must supply the topic name, but
> > supplying a non-default timeout is optional.
> >
> > > I wonder if it would be simpler to just have a
> > > single parameter. In that case it should probably be called a Request as
> > > Radai suggested, but that's a separate point and we can discuss it
> > > separately.
> >
> > Hmm.  I don't think it would be simpler for users.  It would force
> > people who just want to do something simple like delete a topic or get
> > the api version of a single node to go hunting through the request
> > class.
> >
> > best,
> > Colin
> >
> >
> > >
> > > Ismael
> > >
> > > On Thu, Mar 2, 2017 at 1:58 AM, Colin McCabe  wrote:
> > >
> > > > On Wed, Mar 1, 2017, at 15:52, radai wrote:
> > > > > quick comment on the request objects:
> > > > >
> > > > > i see "abstract class NewTopic" and 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-06 Thread Ismael Juma
Thanks Colin. It seems like you replied to me accidentally instead of the
list, so leaving your reply below for the benefit of others.

Regarding the disadvantage of having to hunt through the request class,
don't people have to do that anyway with the Options classes?

Aside from that, it would be great if the KIP included more detailed
javadoc for each method including information about potential exceptions.
I'm particularly interested in what a user can expect if a create topics
succeeds versus what they can expect if a timeout exception is thrown. It
would be good to consider partial failures as well.

Ismael

On Fri, Mar 3, 2017 at 9:37 PM, Colin McCabe  wrote:

> On Fri, Mar 3, 2017, at 06:41, Ismael Juma wrote:
> > Hi Colin,
> >
> > I still need to do a detailed review, but I have a couple of
> > comments/questions:
> >
> > 1. I am not sure about having the options/response classes as inner
> > classes
> > of the interface. It means that file containing the interface will be
> > huge
> > eventually. And the classes are not necessarily related either. Why not
> > use
> > a separate package for them?
>
> Yeah, I think it's reasonable to make these top-level classes and put
> them in separate files.  We can put them all in
> org.apache.kafka.clients.admin.
>
> >
> > 2. Can you elaborate on how one decides one goes in the Options class
> > versus the first parameter?
>
> I guess I think of options as things that you don't have to set.  For
> example, when deleting a topic, you must supply the topic name, but
> supplying a non-default timeout is optional.
>
> > I wonder if it would be simpler to just have a
> > single parameter. In that case it should probably be called a Request as
> > Radai suggested, but that's a separate point and we can discuss it
> > separately.
>
> Hmm.  I don't think it would be simpler for users.  It would force
> people who just want to do something simple like delete a topic or get
> the api version of a single node to go hunting through the request
> class.
>
> best,
> Colin
>
>
> >
> > Ismael
> >
> > On Thu, Mar 2, 2017 at 1:58 AM, Colin McCabe  wrote:
> >
> > > On Wed, Mar 1, 2017, at 15:52, radai wrote:
> > > > quick comment on the request objects:
> > > >
> > > > i see "abstract class NewTopic" and "class NewTopicWithReplication"
> and "
> > > > NewTopicWithReplicaAssignments"
> > > >
> > > > 1. since the result object is called CreateTopicResults should these
> be
> > > > called *Request?
> > >
> > > Hi radai,
> > >
> > > Thanks for taking a look.
> > >
> > > I think using the name "request" would be very confusing here, because
> > > we have a whole family of internal Request classes such as
> > > CreateTopicsRequest, TopicMetataRequest, etc. which are used for RPCs.
> > >
> > > > 2. this seems like a suboptimal approach to me. imagine we add a
> > > > NewTopicWithSecurity, and then we would need
> > > > NewTopicWithReplicationAndSecurity? (or any composable "traits").
> > > > this wont really scale. Wouldnt it be better to have a single (rather
> > > complicated)
> > > > CreateTopicRequest, and use a builder pattern to deal with the
> compexity
> > > > and options? like so:
> > > >
> > > > CreateTopicRequest req =
> > > > AdminRequests.newTopic("bob").replicationFactor(2).
> > > withPartitionAssignment(1,
> > > > "boker7", "broker10").withOption(...).build();
> > > >
> > > > the builder would validate any potentially conflicting options and
> would
> > > > allow piling on the complexity in a more manageable way (note - my
> code
> > > > above intends to demonstrate both a general replication factor and a
> > > > specific assignment for a partiocular partition of that topic, which
> may
> > > > be
> > > > too much freedom).
> > >
> > > We don't need to express every optional bell and whistle by creating a
> > > subclass.  In fact, the proposal already had setConfigs() in the base
> > > class, since it applies to every new topic creation.
> > >
> > > Thinking about it a little more, though, the subclasses don't really
> add
> > > that much value, so we should probably just have NewTopic and no
> > > subclasses.  I removed the subclasses.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > On Wed, Mar 1, 2017 at 1:58 PM, Colin McCabe 
> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > Thanks for commenting, everyone.  Does anyone have more questions
> or
> > > > > comments, or should we vote?  The latest proposal is up at
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 117%3A+Add+a+public+
> > > > > AdminClient+API+for+Kafka+admin+operations
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Thu, Feb 16, 2017, at 15:00, Colin McCabe wrote:
> > > > > > On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> > > > > > > Hey Colin,
> > > > > > >
> > > > > > > Thanks for the update. I have two comments:
> > > > > > >
> > > > > > > - I actually think it is simpler and good 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-03 Thread Colin McCabe
On Fri, Mar 3, 2017, at 06:41, Ismael Juma wrote:
> Hi Colin,
> 
> I still need to do a detailed review, but I have a couple of
> comments/questions:
> 
> 1. I am not sure about having the options/response classes as inner
> classes
> of the interface. It means that file containing the interface will be
> huge
> eventually. And the classes are not necessarily related either. Why not
> use
> a separate package for them?

Yeah, I think it's reasonable to make these top-level classes and put
them in separate files.  We can put them all in
org.apache.kafka.clients.admin.

> 2. Can you elaborate on how one decides one goes in the Options class
> versus the first parameter? 

I guess I think of options as things that you don't have to set.  For
example, when deleting a topic, you must supply the topic name, but
supplying a non-default timeout is optional.

> I wonder if it would be simpler to just have
> a single parameter.

Hmm.  I don't think it would be simpler for users.  It would force
people who just want to do something simple like delete a topic or get
the api version of a single node to go hunting through the request
class.

best,
Colin

> In that case it should probably be called a Request as
> Radai suggested, but that's a separate point and we can discuss it
> separately.
> 
> Ismael
> 
> On Thu, Mar 2, 2017 at 1:58 AM, Colin McCabe  wrote:
> 
> > On Wed, Mar 1, 2017, at 15:52, radai wrote:
> > > quick comment on the request objects:
> > >
> > > i see "abstract class NewTopic" and "class NewTopicWithReplication" and "
> > > NewTopicWithReplicaAssignments"
> > >
> > > 1. since the result object is called CreateTopicResults should these be
> > > called *Request?
> >
> > Hi radai,
> >
> > Thanks for taking a look.
> >
> > I think using the name "request" would be very confusing here, because
> > we have a whole family of internal Request classes such as
> > CreateTopicsRequest, TopicMetataRequest, etc. which are used for RPCs.
> >
> > > 2. this seems like a suboptimal approach to me. imagine we add a
> > > NewTopicWithSecurity, and then we would need
> > > NewTopicWithReplicationAndSecurity? (or any composable "traits").
> > > this wont really scale. Wouldnt it be better to have a single (rather
> > complicated)
> > > CreateTopicRequest, and use a builder pattern to deal with the compexity
> > > and options? like so:
> > >
> > > CreateTopicRequest req =
> > > AdminRequests.newTopic("bob").replicationFactor(2).
> > withPartitionAssignment(1,
> > > "boker7", "broker10").withOption(...).build();
> > >
> > > the builder would validate any potentially conflicting options and would
> > > allow piling on the complexity in a more manageable way (note - my code
> > > above intends to demonstrate both a general replication factor and a
> > > specific assignment for a partiocular partition of that topic, which may
> > > be
> > > too much freedom).
> >
> > We don't need to express every optional bell and whistle by creating a
> > subclass.  In fact, the proposal already had setConfigs() in the base
> > class, since it applies to every new topic creation.
> >
> > Thinking about it a little more, though, the subclasses don't really add
> > that much value, so we should probably just have NewTopic and no
> > subclasses.  I removed the subclasses.
> >
> > best,
> > Colin
> >
> > >
> > > On Wed, Mar 1, 2017 at 1:58 PM, Colin McCabe  wrote:
> > >
> > > > Hi all,
> > > >
> > > > Thanks for commenting, everyone.  Does anyone have more questions or
> > > > comments, or should we vote?  The latest proposal is up at
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 117%3A+Add+a+public+
> > > > AdminClient+API+for+Kafka+admin+operations
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Thu, Feb 16, 2017, at 15:00, Colin McCabe wrote:
> > > > > On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> > > > > > Hey Colin,
> > > > > >
> > > > > > Thanks for the update. I have two comments:
> > > > > >
> > > > > > - I actually think it is simpler and good enough to have per-topic
> > API
> > > > > > instead of batch-of-topic API. This is different from the argument
> > for
> > > > > > batch-of-partition API because, unlike operation on topic, people
> > > > usually
> > > > > > operate on multiple partitions (e.g. seek(), purge()) at a time. Is
> > > > there
> > > > > > performance concern with per-topic API? I am wondering if we
> > should do
> > > > > > per-topic API until there is use-case or performance benefits of
> > > > > > batch-of-topic API.
> > > > >
> > > > > Yes, there is a performance concern with only supporting operations
> > on
> > > > > one topic at a time.  Jay expressed this in some of his earlier
> > emails
> > > > > and some other people did as well.  We have cases in mind for
> > management
> > > > > software where many topics are created at once.
> > > > >
> > > > > >
> > > > > > - Currently we have interface "Consumer" and "Producer". And we
> > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-03 Thread Ismael Juma
Hi Colin,

I still need to do a detailed review, but I have a couple of
comments/questions:

1. I am not sure about having the options/response classes as inner classes
of the interface. It means that file containing the interface will be huge
eventually. And the classes are not necessarily related either. Why not use
a separate package for them?

2. Can you elaborate on how one decides one goes in the Options class
versus the first parameter? I wonder if it would be simpler to just have a
single parameter. In that case it should probably be called a Request as
Radai suggested, but that's a separate point and we can discuss it
separately.

Ismael

On Thu, Mar 2, 2017 at 1:58 AM, Colin McCabe  wrote:

> On Wed, Mar 1, 2017, at 15:52, radai wrote:
> > quick comment on the request objects:
> >
> > i see "abstract class NewTopic" and "class NewTopicWithReplication" and "
> > NewTopicWithReplicaAssignments"
> >
> > 1. since the result object is called CreateTopicResults should these be
> > called *Request?
>
> Hi radai,
>
> Thanks for taking a look.
>
> I think using the name "request" would be very confusing here, because
> we have a whole family of internal Request classes such as
> CreateTopicsRequest, TopicMetataRequest, etc. which are used for RPCs.
>
> > 2. this seems like a suboptimal approach to me. imagine we add a
> > NewTopicWithSecurity, and then we would need
> > NewTopicWithReplicationAndSecurity? (or any composable "traits").
> > this wont really scale. Wouldnt it be better to have a single (rather
> complicated)
> > CreateTopicRequest, and use a builder pattern to deal with the compexity
> > and options? like so:
> >
> > CreateTopicRequest req =
> > AdminRequests.newTopic("bob").replicationFactor(2).
> withPartitionAssignment(1,
> > "boker7", "broker10").withOption(...).build();
> >
> > the builder would validate any potentially conflicting options and would
> > allow piling on the complexity in a more manageable way (note - my code
> > above intends to demonstrate both a general replication factor and a
> > specific assignment for a partiocular partition of that topic, which may
> > be
> > too much freedom).
>
> We don't need to express every optional bell and whistle by creating a
> subclass.  In fact, the proposal already had setConfigs() in the base
> class, since it applies to every new topic creation.
>
> Thinking about it a little more, though, the subclasses don't really add
> that much value, so we should probably just have NewTopic and no
> subclasses.  I removed the subclasses.
>
> best,
> Colin
>
> >
> > On Wed, Mar 1, 2017 at 1:58 PM, Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > Thanks for commenting, everyone.  Does anyone have more questions or
> > > comments, or should we vote?  The latest proposal is up at
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 117%3A+Add+a+public+
> > > AdminClient+API+for+Kafka+admin+operations
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Thu, Feb 16, 2017, at 15:00, Colin McCabe wrote:
> > > > On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> > > > > Hey Colin,
> > > > >
> > > > > Thanks for the update. I have two comments:
> > > > >
> > > > > - I actually think it is simpler and good enough to have per-topic
> API
> > > > > instead of batch-of-topic API. This is different from the argument
> for
> > > > > batch-of-partition API because, unlike operation on topic, people
> > > usually
> > > > > operate on multiple partitions (e.g. seek(), purge()) at a time. Is
> > > there
> > > > > performance concern with per-topic API? I am wondering if we
> should do
> > > > > per-topic API until there is use-case or performance benefits of
> > > > > batch-of-topic API.
> > > >
> > > > Yes, there is a performance concern with only supporting operations
> on
> > > > one topic at a time.  Jay expressed this in some of his earlier
> emails
> > > > and some other people did as well.  We have cases in mind for
> management
> > > > software where many topics are created at once.
> > > >
> > > > >
> > > > > - Currently we have interface "Consumer" and "Producer". And we
> also
> > > have
> > > > > implementations of these two interfaces as "KafkaConsumer" and
> > > > > "KafkaProducer". If we follow the same naming pattern, should we
> have
> > > > > interface "AdminClient" and the implementation "KafkaAdminClient",
> > > > > instead
> > > > > of the other way around?
> > > >
> > > > That's a good point.  We should do that for consistency.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Dong
> > > > >
> > > > >
> > > > > On Thu, Feb 16, 2017 at 10:19 AM, Colin McCabe  >
> > > > > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > So I think people have made some very good points so far.  There
> > > seems
> > > > > > to be agreement that we need to have explicit batch APIs for the
> > > sake of
> > > > > > efficiency, so I added that back.
> > > > > >
> > > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-01 Thread Colin McCabe
On Wed, Mar 1, 2017, at 15:52, radai wrote:
> quick comment on the request objects:
> 
> i see "abstract class NewTopic" and "class NewTopicWithReplication" and "
> NewTopicWithReplicaAssignments"
> 
> 1. since the result object is called CreateTopicResults should these be
> called *Request?

Hi radai,

Thanks for taking a look.

I think using the name "request" would be very confusing here, because
we have a whole family of internal Request classes such as
CreateTopicsRequest, TopicMetataRequest, etc. which are used for RPCs.

> 2. this seems like a suboptimal approach to me. imagine we add a
> NewTopicWithSecurity, and then we would need
> NewTopicWithReplicationAndSecurity? (or any composable "traits").
> this wont really scale. Wouldnt it be better to have a single (rather 
> complicated)
> CreateTopicRequest, and use a builder pattern to deal with the compexity
> and options? like so:
> 
> CreateTopicRequest req =
> AdminRequests.newTopic("bob").replicationFactor(2).withPartitionAssignment(1,
> "boker7", "broker10").withOption(...).build();
> 
> the builder would validate any potentially conflicting options and would
> allow piling on the complexity in a more manageable way (note - my code
> above intends to demonstrate both a general replication factor and a
> specific assignment for a partiocular partition of that topic, which may
> be
> too much freedom).

We don't need to express every optional bell and whistle by creating a
subclass.  In fact, the proposal already had setConfigs() in the base
class, since it applies to every new topic creation.

Thinking about it a little more, though, the subclasses don't really add
that much value, so we should probably just have NewTopic and no
subclasses.  I removed the subclasses.

best,
Colin

> 
> On Wed, Mar 1, 2017 at 1:58 PM, Colin McCabe  wrote:
> 
> > Hi all,
> >
> > Thanks for commenting, everyone.  Does anyone have more questions or
> > comments, or should we vote?  The latest proposal is up at
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public+
> > AdminClient+API+for+Kafka+admin+operations
> >
> > best,
> > Colin
> >
> >
> > On Thu, Feb 16, 2017, at 15:00, Colin McCabe wrote:
> > > On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> > > > Hey Colin,
> > > >
> > > > Thanks for the update. I have two comments:
> > > >
> > > > - I actually think it is simpler and good enough to have per-topic API
> > > > instead of batch-of-topic API. This is different from the argument for
> > > > batch-of-partition API because, unlike operation on topic, people
> > usually
> > > > operate on multiple partitions (e.g. seek(), purge()) at a time. Is
> > there
> > > > performance concern with per-topic API? I am wondering if we should do
> > > > per-topic API until there is use-case or performance benefits of
> > > > batch-of-topic API.
> > >
> > > Yes, there is a performance concern with only supporting operations on
> > > one topic at a time.  Jay expressed this in some of his earlier emails
> > > and some other people did as well.  We have cases in mind for management
> > > software where many topics are created at once.
> > >
> > > >
> > > > - Currently we have interface "Consumer" and "Producer". And we also
> > have
> > > > implementations of these two interfaces as "KafkaConsumer" and
> > > > "KafkaProducer". If we follow the same naming pattern, should we have
> > > > interface "AdminClient" and the implementation "KafkaAdminClient",
> > > > instead
> > > > of the other way around?
> > >
> > > That's a good point.  We should do that for consistency.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Dong
> > > >
> > > >
> > > > On Thu, Feb 16, 2017 at 10:19 AM, Colin McCabe 
> > > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > So I think people have made some very good points so far.  There
> > seems
> > > > > to be agreement that we need to have explicit batch APIs for the
> > sake of
> > > > > efficiency, so I added that back.
> > > > >
> > > > > Contexts seem a little more complex than we thought, so I removed
> > that
> > > > > from the proposal.
> > > > >
> > > > > I removed the Impl class.  Instead, we now have a KafkaAdminClient
> > > > > interface and an AdminClient implementation.  I think this matches
> > our
> > > > > other code better, as Jay commented.
> > > > >
> > > > > Each call now has an "Options" object that is passed in.  This will
> > > > > allow us to easily add new parameters to the calls without having
> > tons
> > > > > of function overloads.  Similarly, each call now has a Results
> > object,
> > > > > which will let us easily extend the results we are returning if
> > needed.
> > > > >
> > > > > Many people made the point that Java 7 Futures are not that useful,
> > but
> > > > > Java 8 CompletableFutures are.  With CompletableFutures, you can
> > chain
> > > > > calls, adapt them, join them-- basically all the stuff people are
> > doing
> > > > > in Node.js 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-01 Thread radai
quick comment on the request objects:

i see "abstract class NewTopic" and "class NewTopicWithReplication" and "
NewTopicWithReplicaAssignments"

1. since the result object is called CreateTopicResults should these be
called *Request?
2. this seems like a suboptimal approach to me. imagine we add a
NewTopicWithSecurity, and then we would need
NewTopicWithReplicationAndSecurity? (or any composable "traits"). this wont
really scale. Wouldnt it be better to have a single (rather complicated)
CreateTopicRequest, and use a builder pattern to deal with the compexity
and options? like so:

CreateTopicRequest req =
AdminRequests.newTopic("bob").replicationFactor(2).withPartitionAssignment(1,
"boker7", "broker10").withOption(...).build();

the builder would validate any potentially conflicting options and would
allow piling on the complexity in a more manageable way (note - my code
above intends to demonstrate both a general replication factor and a
specific assignment for a partiocular partition of that topic, which may be
too much freedom).

On Wed, Mar 1, 2017 at 1:58 PM, Colin McCabe  wrote:

> Hi all,
>
> Thanks for commenting, everyone.  Does anyone have more questions or
> comments, or should we vote?  The latest proposal is up at
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public+
> AdminClient+API+for+Kafka+admin+operations
>
> best,
> Colin
>
>
> On Thu, Feb 16, 2017, at 15:00, Colin McCabe wrote:
> > On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> > > Hey Colin,
> > >
> > > Thanks for the update. I have two comments:
> > >
> > > - I actually think it is simpler and good enough to have per-topic API
> > > instead of batch-of-topic API. This is different from the argument for
> > > batch-of-partition API because, unlike operation on topic, people
> usually
> > > operate on multiple partitions (e.g. seek(), purge()) at a time. Is
> there
> > > performance concern with per-topic API? I am wondering if we should do
> > > per-topic API until there is use-case or performance benefits of
> > > batch-of-topic API.
> >
> > Yes, there is a performance concern with only supporting operations on
> > one topic at a time.  Jay expressed this in some of his earlier emails
> > and some other people did as well.  We have cases in mind for management
> > software where many topics are created at once.
> >
> > >
> > > - Currently we have interface "Consumer" and "Producer". And we also
> have
> > > implementations of these two interfaces as "KafkaConsumer" and
> > > "KafkaProducer". If we follow the same naming pattern, should we have
> > > interface "AdminClient" and the implementation "KafkaAdminClient",
> > > instead
> > > of the other way around?
> >
> > That's a good point.  We should do that for consistency.
> >
> > best,
> > Colin
> >
> > >
> > > Dong
> > >
> > >
> > > On Thu, Feb 16, 2017 at 10:19 AM, Colin McCabe 
> > > wrote:
> > >
> > > > Hi all,
> > > >
> > > > So I think people have made some very good points so far.  There
> seems
> > > > to be agreement that we need to have explicit batch APIs for the
> sake of
> > > > efficiency, so I added that back.
> > > >
> > > > Contexts seem a little more complex than we thought, so I removed
> that
> > > > from the proposal.
> > > >
> > > > I removed the Impl class.  Instead, we now have a KafkaAdminClient
> > > > interface and an AdminClient implementation.  I think this matches
> our
> > > > other code better, as Jay commented.
> > > >
> > > > Each call now has an "Options" object that is passed in.  This will
> > > > allow us to easily add new parameters to the calls without having
> tons
> > > > of function overloads.  Similarly, each call now has a Results
> object,
> > > > which will let us easily extend the results we are returning if
> needed.
> > > >
> > > > Many people made the point that Java 7 Futures are not that useful,
> but
> > > > Java 8 CompletableFutures are.  With CompletableFutures, you can
> chain
> > > > calls, adapt them, join them-- basically all the stuff people are
> doing
> > > > in Node.js and Twisted Python.  Java 7 Futures don't really let you
> do
> > > > anything but poll for a value or block.  So I felt that it was
> better to
> > > > just go with a CompletableFuture-based API.
> > > >
> > > > People also made the point that they would like an easy API for
> waiting
> > > > on complete success of a batch call.  For example, an API that would
> > > > fail if even one topic wasn't created in createTopics.  So I came up
> > > > with Result objects that provide multiple futures that you can wait
> on.
> > > > You can wait on a future that fires when everything is complete, or
> you
> > > > can wait on futures for individual topics being created.
> > > >
> > > > I updated the wiki, so please take a look.  Note that this new API
> > > > requires JDK8.  It seems like JDK8 is coming soon, though, and the
> > > > disadvantages of sticking to Java 7 are pretty big here, I 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-01 Thread Colin McCabe
Hi all,

Thanks for commenting, everyone.  Does anyone have more questions or
comments, or should we vote?  The latest proposal is up at
https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public+AdminClient+API+for+Kafka+admin+operations

best,
Colin


On Thu, Feb 16, 2017, at 15:00, Colin McCabe wrote:
> On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> > Hey Colin,
> > 
> > Thanks for the update. I have two comments:
> > 
> > - I actually think it is simpler and good enough to have per-topic API
> > instead of batch-of-topic API. This is different from the argument for
> > batch-of-partition API because, unlike operation on topic, people usually
> > operate on multiple partitions (e.g. seek(), purge()) at a time. Is there
> > performance concern with per-topic API? I am wondering if we should do
> > per-topic API until there is use-case or performance benefits of
> > batch-of-topic API.
> 
> Yes, there is a performance concern with only supporting operations on
> one topic at a time.  Jay expressed this in some of his earlier emails
> and some other people did as well.  We have cases in mind for management
> software where many topics are created at once.
> 
> > 
> > - Currently we have interface "Consumer" and "Producer". And we also have
> > implementations of these two interfaces as "KafkaConsumer" and
> > "KafkaProducer". If we follow the same naming pattern, should we have
> > interface "AdminClient" and the implementation "KafkaAdminClient",
> > instead
> > of the other way around?
> 
> That's a good point.  We should do that for consistency.
> 
> best,
> Colin
> 
> > 
> > Dong
> > 
> > 
> > On Thu, Feb 16, 2017 at 10:19 AM, Colin McCabe 
> > wrote:
> > 
> > > Hi all,
> > >
> > > So I think people have made some very good points so far.  There seems
> > > to be agreement that we need to have explicit batch APIs for the sake of
> > > efficiency, so I added that back.
> > >
> > > Contexts seem a little more complex than we thought, so I removed that
> > > from the proposal.
> > >
> > > I removed the Impl class.  Instead, we now have a KafkaAdminClient
> > > interface and an AdminClient implementation.  I think this matches our
> > > other code better, as Jay commented.
> > >
> > > Each call now has an "Options" object that is passed in.  This will
> > > allow us to easily add new parameters to the calls without having tons
> > > of function overloads.  Similarly, each call now has a Results object,
> > > which will let us easily extend the results we are returning if needed.
> > >
> > > Many people made the point that Java 7 Futures are not that useful, but
> > > Java 8 CompletableFutures are.  With CompletableFutures, you can chain
> > > calls, adapt them, join them-- basically all the stuff people are doing
> > > in Node.js and Twisted Python.  Java 7 Futures don't really let you do
> > > anything but poll for a value or block.  So I felt that it was better to
> > > just go with a CompletableFuture-based API.
> > >
> > > People also made the point that they would like an easy API for waiting
> > > on complete success of a batch call.  For example, an API that would
> > > fail if even one topic wasn't created in createTopics.  So I came up
> > > with Result objects that provide multiple futures that you can wait on.
> > > You can wait on a future that fires when everything is complete, or you
> > > can wait on futures for individual topics being created.
> > >
> > > I updated the wiki, so please take a look.  Note that this new API
> > > requires JDK8.  It seems like JDK8 is coming soon, though, and the
> > > disadvantages of sticking to Java 7 are pretty big here, I think.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Mon, Feb 13, 2017, at 11:51, Colin McCabe wrote:
> > > > On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote:
> > > > > Hey Colin,
> > > > >
> > > > > Thanks for the hard work on this. I know going back and forth on APIs
> > > is
> > > > > kind of frustrating but we're at the point where these things live 
> > > > > long
> > > > > enough and are used by enough people that it is worth the pain. I'm
> > > sure
> > > > > it'll come down in the right place eventually. A couple things I've
> > > found
> > > > > helped in the past:
> > > > >
> > > > >1. The burden of evidence needs to fall on the complicator. i.e. if
> > > > >person X thinks the api should be async they need to produce a set
> > > of
> > > > >common use cases that require this. Otherwise you are perpetually
> > > > >having to
> > > > >think "we might need x". I think it is good to have a rule of
> > > "simple
> > > > >until
> > > > >proven insufficient".
> > > > >2. Make sure we frame things for the intended audience. At this
> > > point
> > > > >our apis get used by a very broad set of Java engineers. This is a
> > > > >very
> > > > >different audience from our developer mailing list. These people
> > > code
> > > > >for a
> > > > >

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-16 Thread Colin McCabe
On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> Hey Colin,
> 
> Thanks for the update. I have two comments:
> 
> - I actually think it is simpler and good enough to have per-topic API
> instead of batch-of-topic API. This is different from the argument for
> batch-of-partition API because, unlike operation on topic, people usually
> operate on multiple partitions (e.g. seek(), purge()) at a time. Is there
> performance concern with per-topic API? I am wondering if we should do
> per-topic API until there is use-case or performance benefits of
> batch-of-topic API.

Yes, there is a performance concern with only supporting operations on
one topic at a time.  Jay expressed this in some of his earlier emails
and some other people did as well.  We have cases in mind for management
software where many topics are created at once.

> 
> - Currently we have interface "Consumer" and "Producer". And we also have
> implementations of these two interfaces as "KafkaConsumer" and
> "KafkaProducer". If we follow the same naming pattern, should we have
> interface "AdminClient" and the implementation "KafkaAdminClient",
> instead
> of the other way around?

That's a good point.  We should do that for consistency.

best,
Colin

> 
> Dong
> 
> 
> On Thu, Feb 16, 2017 at 10:19 AM, Colin McCabe 
> wrote:
> 
> > Hi all,
> >
> > So I think people have made some very good points so far.  There seems
> > to be agreement that we need to have explicit batch APIs for the sake of
> > efficiency, so I added that back.
> >
> > Contexts seem a little more complex than we thought, so I removed that
> > from the proposal.
> >
> > I removed the Impl class.  Instead, we now have a KafkaAdminClient
> > interface and an AdminClient implementation.  I think this matches our
> > other code better, as Jay commented.
> >
> > Each call now has an "Options" object that is passed in.  This will
> > allow us to easily add new parameters to the calls without having tons
> > of function overloads.  Similarly, each call now has a Results object,
> > which will let us easily extend the results we are returning if needed.
> >
> > Many people made the point that Java 7 Futures are not that useful, but
> > Java 8 CompletableFutures are.  With CompletableFutures, you can chain
> > calls, adapt them, join them-- basically all the stuff people are doing
> > in Node.js and Twisted Python.  Java 7 Futures don't really let you do
> > anything but poll for a value or block.  So I felt that it was better to
> > just go with a CompletableFuture-based API.
> >
> > People also made the point that they would like an easy API for waiting
> > on complete success of a batch call.  For example, an API that would
> > fail if even one topic wasn't created in createTopics.  So I came up
> > with Result objects that provide multiple futures that you can wait on.
> > You can wait on a future that fires when everything is complete, or you
> > can wait on futures for individual topics being created.
> >
> > I updated the wiki, so please take a look.  Note that this new API
> > requires JDK8.  It seems like JDK8 is coming soon, though, and the
> > disadvantages of sticking to Java 7 are pretty big here, I think.
> >
> > best,
> > Colin
> >
> >
> > On Mon, Feb 13, 2017, at 11:51, Colin McCabe wrote:
> > > On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote:
> > > > Hey Colin,
> > > >
> > > > Thanks for the hard work on this. I know going back and forth on APIs
> > is
> > > > kind of frustrating but we're at the point where these things live long
> > > > enough and are used by enough people that it is worth the pain. I'm
> > sure
> > > > it'll come down in the right place eventually. A couple things I've
> > found
> > > > helped in the past:
> > > >
> > > >1. The burden of evidence needs to fall on the complicator. i.e. if
> > > >person X thinks the api should be async they need to produce a set
> > of
> > > >common use cases that require this. Otherwise you are perpetually
> > > >having to
> > > >think "we might need x". I think it is good to have a rule of
> > "simple
> > > >until
> > > >proven insufficient".
> > > >2. Make sure we frame things for the intended audience. At this
> > point
> > > >our apis get used by a very broad set of Java engineers. This is a
> > > >very
> > > >different audience from our developer mailing list. These people
> > code
> > > >for a
> > > >living not necessarily as a passion, and may not understand details
> > of
> > > >the
> > > >internals of our system or even basic things like multi-threaded
> > > >programming. I don't think this means we want to dumb things down,
> > but
> > > >rather try really hard to make things truly simple when possible.
> > > >
> > > > Okay here were a couple of comments:
> > > >
> > > >1. Conceptually what is a TopicContext? I think it means something
> > > >like
> > > >TopicAdmin? It is not literally context about Topics 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-16 Thread Dong Lin
Hey Colin,

Thanks for the update. I have two comments:

- I actually think it is simpler and good enough to have per-topic API
instead of batch-of-topic API. This is different from the argument for
batch-of-partition API because, unlike operation on topic, people usually
operate on multiple partitions (e.g. seek(), purge()) at a time. Is there
performance concern with per-topic API? I am wondering if we should do
per-topic API until there is use-case or performance benefits of
batch-of-topic API.

- Currently we have interface "Consumer" and "Producer". And we also have
implementations of these two interfaces as "KafkaConsumer" and
"KafkaProducer". If we follow the same naming pattern, should we have
interface "AdminClient" and the implementation "KafkaAdminClient", instead
of the other way around?

Dong


On Thu, Feb 16, 2017 at 10:19 AM, Colin McCabe  wrote:

> Hi all,
>
> So I think people have made some very good points so far.  There seems
> to be agreement that we need to have explicit batch APIs for the sake of
> efficiency, so I added that back.
>
> Contexts seem a little more complex than we thought, so I removed that
> from the proposal.
>
> I removed the Impl class.  Instead, we now have a KafkaAdminClient
> interface and an AdminClient implementation.  I think this matches our
> other code better, as Jay commented.
>
> Each call now has an "Options" object that is passed in.  This will
> allow us to easily add new parameters to the calls without having tons
> of function overloads.  Similarly, each call now has a Results object,
> which will let us easily extend the results we are returning if needed.
>
> Many people made the point that Java 7 Futures are not that useful, but
> Java 8 CompletableFutures are.  With CompletableFutures, you can chain
> calls, adapt them, join them-- basically all the stuff people are doing
> in Node.js and Twisted Python.  Java 7 Futures don't really let you do
> anything but poll for a value or block.  So I felt that it was better to
> just go with a CompletableFuture-based API.
>
> People also made the point that they would like an easy API for waiting
> on complete success of a batch call.  For example, an API that would
> fail if even one topic wasn't created in createTopics.  So I came up
> with Result objects that provide multiple futures that you can wait on.
> You can wait on a future that fires when everything is complete, or you
> can wait on futures for individual topics being created.
>
> I updated the wiki, so please take a look.  Note that this new API
> requires JDK8.  It seems like JDK8 is coming soon, though, and the
> disadvantages of sticking to Java 7 are pretty big here, I think.
>
> best,
> Colin
>
>
> On Mon, Feb 13, 2017, at 11:51, Colin McCabe wrote:
> > On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote:
> > > Hey Colin,
> > >
> > > Thanks for the hard work on this. I know going back and forth on APIs
> is
> > > kind of frustrating but we're at the point where these things live long
> > > enough and are used by enough people that it is worth the pain. I'm
> sure
> > > it'll come down in the right place eventually. A couple things I've
> found
> > > helped in the past:
> > >
> > >1. The burden of evidence needs to fall on the complicator. i.e. if
> > >person X thinks the api should be async they need to produce a set
> of
> > >common use cases that require this. Otherwise you are perpetually
> > >having to
> > >think "we might need x". I think it is good to have a rule of
> "simple
> > >until
> > >proven insufficient".
> > >2. Make sure we frame things for the intended audience. At this
> point
> > >our apis get used by a very broad set of Java engineers. This is a
> > >very
> > >different audience from our developer mailing list. These people
> code
> > >for a
> > >living not necessarily as a passion, and may not understand details
> of
> > >the
> > >internals of our system or even basic things like multi-threaded
> > >programming. I don't think this means we want to dumb things down,
> but
> > >rather try really hard to make things truly simple when possible.
> > >
> > > Okay here were a couple of comments:
> > >
> > >1. Conceptually what is a TopicContext? I think it means something
> > >like
> > >TopicAdmin? It is not literally context about Topics right? What is
> > >the
> > >relationship of Contexts to clients? Is there a threadsafety
> > >difference?
> > >Would be nice to not have to think about this, this is what I mean
> by
> > >"conceptual weight". We introduce a new concept that is a bit
> nebulous
> > >that
> > >I have to figure out to use what could be a simple api. I'm sure
> > >you've
> > >been through this experience before where you have these various
> > >objects
> > >and you're trying to figure out what they represent (the connection
> to
> > >the
> > >server? the 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-16 Thread Colin McCabe
Hi all,

So I think people have made some very good points so far.  There seems
to be agreement that we need to have explicit batch APIs for the sake of
efficiency, so I added that back.

Contexts seem a little more complex than we thought, so I removed that
from the proposal.

I removed the Impl class.  Instead, we now have a KafkaAdminClient
interface and an AdminClient implementation.  I think this matches our
other code better, as Jay commented.

Each call now has an "Options" object that is passed in.  This will
allow us to easily add new parameters to the calls without having tons
of function overloads.  Similarly, each call now has a Results object,
which will let us easily extend the results we are returning if needed.

Many people made the point that Java 7 Futures are not that useful, but
Java 8 CompletableFutures are.  With CompletableFutures, you can chain
calls, adapt them, join them-- basically all the stuff people are doing
in Node.js and Twisted Python.  Java 7 Futures don't really let you do
anything but poll for a value or block.  So I felt that it was better to
just go with a CompletableFuture-based API.

People also made the point that they would like an easy API for waiting
on complete success of a batch call.  For example, an API that would
fail if even one topic wasn't created in createTopics.  So I came up
with Result objects that provide multiple futures that you can wait on. 
You can wait on a future that fires when everything is complete, or you
can wait on futures for individual topics being created.

I updated the wiki, so please take a look.  Note that this new API
requires JDK8.  It seems like JDK8 is coming soon, though, and the
disadvantages of sticking to Java 7 are pretty big here, I think.

best,
Colin


On Mon, Feb 13, 2017, at 11:51, Colin McCabe wrote:
> On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote:
> > Hey Colin,
> > 
> > Thanks for the hard work on this. I know going back and forth on APIs is
> > kind of frustrating but we're at the point where these things live long
> > enough and are used by enough people that it is worth the pain. I'm sure
> > it'll come down in the right place eventually. A couple things I've found
> > helped in the past:
> > 
> >1. The burden of evidence needs to fall on the complicator. i.e. if
> >person X thinks the api should be async they need to produce a set of
> >common use cases that require this. Otherwise you are perpetually
> >having to
> >think "we might need x". I think it is good to have a rule of "simple
> >until
> >proven insufficient".
> >2. Make sure we frame things for the intended audience. At this point
> >our apis get used by a very broad set of Java engineers. This is a
> >very
> >different audience from our developer mailing list. These people code
> >for a
> >living not necessarily as a passion, and may not understand details of
> >the
> >internals of our system or even basic things like multi-threaded
> >programming. I don't think this means we want to dumb things down, but
> >rather try really hard to make things truly simple when possible.
> > 
> > Okay here were a couple of comments:
> > 
> >1. Conceptually what is a TopicContext? I think it means something
> >like
> >TopicAdmin? It is not literally context about Topics right? What is
> >the
> >relationship of Contexts to clients? Is there a threadsafety
> >difference?
> >Would be nice to not have to think about this, this is what I mean by
> >"conceptual weight". We introduce a new concept that is a bit nebulous
> >that
> >I have to figure out to use what could be a simple api. I'm sure
> >you've
> >been through this experience before where you have these various
> >objects
> >and you're trying to figure out what they represent (the connection to
> >the
> >server? the information to create a connection? a request session?).
> 
> The intention was to provide some grouping of methods, and also a place
> to put request parameters which were often set to defaults rather than
> being explicitly set.  If it seems complex, we can certainly get rid of
> it.
> 
> >2. We've tried to avoid the Impl naming convention. In general the
> >rule
> >has been if there is only going to be one implementation you don't
> >need an
> >interface. If there will be multiple, distinguish it from the others.
> >The
> >other clients follow this pattern: Producer, KafkaProducer,
> >MockProducer;
> >Consumer, KafkaConsumer, MockConsumer.
> 
> Good point.  Let's change the interface to KafkaAdminClient, and the
> implementation to AdminClient.
> 
> >3. We generally don't use setters or getters as a naming convention. I
> >personally think mutating the setting in place seems kind of like late
> >90s
> >Java style. I think it likely has thread-safety issues. i.e. even if
> >it is
> >volatile you may not get 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-16 Thread Colin McCabe
On Mon, Feb 13, 2017, at 23:04, radai wrote:
> 1. making the client Closeable/AutoCloseable would allow try (Client =
> ...)
> {} without the need to finally close.

Good idea... let's make the interface extend AutoCloseable.

> 
> 2. a "stream processing unit" (producer + consumer) currently holds 2
> open
> sockets to every broker it interacts with, because producer and consumer
> dont share the network stack. if we use the admin API to auto cleanup on
> commit for intermediate pipelines (which is one of our use cases) this
> figure goes up to 3 sockets per unit of processing per broker. beyond
> becoming a scalability issue this (i think) might also introduce annoying
> bugs due to some (but not all) of these connections being down. this is
> not
> an issue of this KIP though.

Right, that is out of scope for this KIP, which is just about public
APIs.

It's worth thinking about for the future, though.

best,
Colin

> 
> On Mon, Feb 13, 2017 at 11:51 AM, Colin McCabe 
> wrote:
> 
> > On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote:
> > > Hey Colin,
> > >
> > > Thanks for the hard work on this. I know going back and forth on APIs is
> > > kind of frustrating but we're at the point where these things live long
> > > enough and are used by enough people that it is worth the pain. I'm sure
> > > it'll come down in the right place eventually. A couple things I've found
> > > helped in the past:
> > >
> > >1. The burden of evidence needs to fall on the complicator. i.e. if
> > >person X thinks the api should be async they need to produce a set of
> > >common use cases that require this. Otherwise you are perpetually
> > >having to
> > >think "we might need x". I think it is good to have a rule of "simple
> > >until
> > >proven insufficient".
> > >2. Make sure we frame things for the intended audience. At this point
> > >our apis get used by a very broad set of Java engineers. This is a
> > >very
> > >different audience from our developer mailing list. These people code
> > >for a
> > >living not necessarily as a passion, and may not understand details of
> > >the
> > >internals of our system or even basic things like multi-threaded
> > >programming. I don't think this means we want to dumb things down, but
> > >rather try really hard to make things truly simple when possible.
> > >
> > > Okay here were a couple of comments:
> > >
> > >1. Conceptually what is a TopicContext? I think it means something
> > >like
> > >TopicAdmin? It is not literally context about Topics right? What is
> > >the
> > >relationship of Contexts to clients? Is there a threadsafety
> > >difference?
> > >Would be nice to not have to think about this, this is what I mean by
> > >"conceptual weight". We introduce a new concept that is a bit nebulous
> > >that
> > >I have to figure out to use what could be a simple api. I'm sure
> > >you've
> > >been through this experience before where you have these various
> > >objects
> > >and you're trying to figure out what they represent (the connection to
> > >the
> > >server? the information to create a connection? a request session?).
> >
> > The intention was to provide some grouping of methods, and also a place
> > to put request parameters which were often set to defaults rather than
> > being explicitly set.  If it seems complex, we can certainly get rid of
> > it.
> >
> > >2. We've tried to avoid the Impl naming convention. In general the
> > >rule
> > >has been if there is only going to be one implementation you don't
> > >need an
> > >interface. If there will be multiple, distinguish it from the others.
> > >The
> > >other clients follow this pattern: Producer, KafkaProducer,
> > >MockProducer;
> > >Consumer, KafkaConsumer, MockConsumer.
> >
> > Good point.  Let's change the interface to KafkaAdminClient, and the
> > implementation to AdminClient.
> >
> > >3. We generally don't use setters or getters as a naming convention. I
> > >personally think mutating the setting in place seems kind of like late
> > >90s
> > >Java style. I think it likely has thread-safety issues. i.e. even if
> > >it is
> > >volatile you may not get the value you just set if there is another
> > >thread... I actually really liked what you described as your original
> > >idea
> > >of having a single parameter object like CreateTopicRequest that holds
> > >all
> > >these parameters and defaults. This lets you evolve the api with all
> > >the
> > >various combinations of arguments without overloading insanity. After
> > >doing
> > >literally tens of thousands of remote APIs at LinkedIn we eventually
> > >converged on a rule, which is ultimately every remote api needs a
> > >single
> > >argument object you can add to over time and it must be batched. Which
> > >

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-13 Thread radai
1. making the client Closeable/AutoCloseable would allow try (Client = ...)
{} without the need to finally close.

2. a "stream processing unit" (producer + consumer) currently holds 2 open
sockets to every broker it interacts with, because producer and consumer
dont share the network stack. if we use the admin API to auto cleanup on
commit for intermediate pipelines (which is one of our use cases) this
figure goes up to 3 sockets per unit of processing per broker. beyond
becoming a scalability issue this (i think) might also introduce annoying
bugs due to some (but not all) of these connections being down. this is not
an issue of this KIP though.

On Mon, Feb 13, 2017 at 11:51 AM, Colin McCabe  wrote:

> On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote:
> > Hey Colin,
> >
> > Thanks for the hard work on this. I know going back and forth on APIs is
> > kind of frustrating but we're at the point where these things live long
> > enough and are used by enough people that it is worth the pain. I'm sure
> > it'll come down in the right place eventually. A couple things I've found
> > helped in the past:
> >
> >1. The burden of evidence needs to fall on the complicator. i.e. if
> >person X thinks the api should be async they need to produce a set of
> >common use cases that require this. Otherwise you are perpetually
> >having to
> >think "we might need x". I think it is good to have a rule of "simple
> >until
> >proven insufficient".
> >2. Make sure we frame things for the intended audience. At this point
> >our apis get used by a very broad set of Java engineers. This is a
> >very
> >different audience from our developer mailing list. These people code
> >for a
> >living not necessarily as a passion, and may not understand details of
> >the
> >internals of our system or even basic things like multi-threaded
> >programming. I don't think this means we want to dumb things down, but
> >rather try really hard to make things truly simple when possible.
> >
> > Okay here were a couple of comments:
> >
> >1. Conceptually what is a TopicContext? I think it means something
> >like
> >TopicAdmin? It is not literally context about Topics right? What is
> >the
> >relationship of Contexts to clients? Is there a threadsafety
> >difference?
> >Would be nice to not have to think about this, this is what I mean by
> >"conceptual weight". We introduce a new concept that is a bit nebulous
> >that
> >I have to figure out to use what could be a simple api. I'm sure
> >you've
> >been through this experience before where you have these various
> >objects
> >and you're trying to figure out what they represent (the connection to
> >the
> >server? the information to create a connection? a request session?).
>
> The intention was to provide some grouping of methods, and also a place
> to put request parameters which were often set to defaults rather than
> being explicitly set.  If it seems complex, we can certainly get rid of
> it.
>
> >2. We've tried to avoid the Impl naming convention. In general the
> >rule
> >has been if there is only going to be one implementation you don't
> >need an
> >interface. If there will be multiple, distinguish it from the others.
> >The
> >other clients follow this pattern: Producer, KafkaProducer,
> >MockProducer;
> >Consumer, KafkaConsumer, MockConsumer.
>
> Good point.  Let's change the interface to KafkaAdminClient, and the
> implementation to AdminClient.
>
> >3. We generally don't use setters or getters as a naming convention. I
> >personally think mutating the setting in place seems kind of like late
> >90s
> >Java style. I think it likely has thread-safety issues. i.e. even if
> >it is
> >volatile you may not get the value you just set if there is another
> >thread... I actually really liked what you described as your original
> >idea
> >of having a single parameter object like CreateTopicRequest that holds
> >all
> >these parameters and defaults. This lets you evolve the api with all
> >the
> >various combinations of arguments without overloading insanity. After
> >doing
> >literally tens of thousands of remote APIs at LinkedIn we eventually
> >converged on a rule, which is ultimately every remote api needs a
> >single
> >argument object you can add to over time and it must be batched. Which
> >brings me to my next point...
>
> Just to clarify, volatiles were never a part of the proposal.  I think
> that context objects or request objects should be used by a single
> thread at a time.
>
> I'm not opposed to request objects, but I think they raise all the same
> questions as context objects.  Basically, the thread-safety issues need
> to be spelled out and understood by the user, and the user needs more
> lines of code to make a request. 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-13 Thread Colin McCabe
On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote:
> Hey Colin,
> 
> Thanks for the hard work on this. I know going back and forth on APIs is
> kind of frustrating but we're at the point where these things live long
> enough and are used by enough people that it is worth the pain. I'm sure
> it'll come down in the right place eventually. A couple things I've found
> helped in the past:
> 
>1. The burden of evidence needs to fall on the complicator. i.e. if
>person X thinks the api should be async they need to produce a set of
>common use cases that require this. Otherwise you are perpetually
>having to
>think "we might need x". I think it is good to have a rule of "simple
>until
>proven insufficient".
>2. Make sure we frame things for the intended audience. At this point
>our apis get used by a very broad set of Java engineers. This is a
>very
>different audience from our developer mailing list. These people code
>for a
>living not necessarily as a passion, and may not understand details of
>the
>internals of our system or even basic things like multi-threaded
>programming. I don't think this means we want to dumb things down, but
>rather try really hard to make things truly simple when possible.
> 
> Okay here were a couple of comments:
> 
>1. Conceptually what is a TopicContext? I think it means something
>like
>TopicAdmin? It is not literally context about Topics right? What is
>the
>relationship of Contexts to clients? Is there a threadsafety
>difference?
>Would be nice to not have to think about this, this is what I mean by
>"conceptual weight". We introduce a new concept that is a bit nebulous
>that
>I have to figure out to use what could be a simple api. I'm sure
>you've
>been through this experience before where you have these various
>objects
>and you're trying to figure out what they represent (the connection to
>the
>server? the information to create a connection? a request session?).

The intention was to provide some grouping of methods, and also a place
to put request parameters which were often set to defaults rather than
being explicitly set.  If it seems complex, we can certainly get rid of
it.

>2. We've tried to avoid the Impl naming convention. In general the
>rule
>has been if there is only going to be one implementation you don't
>need an
>interface. If there will be multiple, distinguish it from the others.
>The
>other clients follow this pattern: Producer, KafkaProducer,
>MockProducer;
>Consumer, KafkaConsumer, MockConsumer.

Good point.  Let's change the interface to KafkaAdminClient, and the
implementation to AdminClient.

>3. We generally don't use setters or getters as a naming convention. I
>personally think mutating the setting in place seems kind of like late
>90s
>Java style. I think it likely has thread-safety issues. i.e. even if
>it is
>volatile you may not get the value you just set if there is another
>thread... I actually really liked what you described as your original
>idea
>of having a single parameter object like CreateTopicRequest that holds
>all
>these parameters and defaults. This lets you evolve the api with all
>the
>various combinations of arguments without overloading insanity. After
>doing
>literally tens of thousands of remote APIs at LinkedIn we eventually
>converged on a rule, which is ultimately every remote api needs a
>single
>argument object you can add to over time and it must be batched. Which
>brings me to my next point...

Just to clarify, volatiles were never a part of the proposal.  I think
that context objects or request objects should be used by a single
thread at a time.

I'm not opposed to request objects, but I think they raise all the same
questions as context objects.  Basically, the thread-safety issues need
to be spelled out and understood by the user, and the user needs more
lines of code to make a request.  And there will be people trying to do
things like re-use request objects when they should not, and so forth.

>4. I agree batch apis are annoying but I suspect we'll end up adding
>one. Doing 1000 requests for 1000 operations if creating or deleting
>will
>be bad, right? This won't be the common case, but when you do it it
>will be
>a deal-breaker problem. I don't think we should try to fix this one
>behind
>the scenes.
>5. Are we going to do CompletableFuture (which requires java 8) or
>normal Future? Normal Future is utterly useless for most things other
>than
>just calling wait. If we can evolve in place from Future to
>CompletableFuture that is fantastic (we could do it for the producer
>too!).
>My belief was that this was binary incompatible but I actually don't
>know
>(obviously it's source compatible).

In my testing, 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-12 Thread Jay Kreps
Hey Colin,

Thanks for the hard work on this. I know going back and forth on APIs is
kind of frustrating but we're at the point where these things live long
enough and are used by enough people that it is worth the pain. I'm sure
it'll come down in the right place eventually. A couple things I've found
helped in the past:

   1. The burden of evidence needs to fall on the complicator. i.e. if
   person X thinks the api should be async they need to produce a set of
   common use cases that require this. Otherwise you are perpetually having to
   think "we might need x". I think it is good to have a rule of "simple until
   proven insufficient".
   2. Make sure we frame things for the intended audience. At this point
   our apis get used by a very broad set of Java engineers. This is a very
   different audience from our developer mailing list. These people code for a
   living not necessarily as a passion, and may not understand details of the
   internals of our system or even basic things like multi-threaded
   programming. I don't think this means we want to dumb things down, but
   rather try really hard to make things truly simple when possible.

Okay here were a couple of comments:

   1. Conceptually what is a TopicContext? I think it means something like
   TopicAdmin? It is not literally context about Topics right? What is the
   relationship of Contexts to clients? Is there a threadsafety difference?
   Would be nice to not have to think about this, this is what I mean by
   "conceptual weight". We introduce a new concept that is a bit nebulous that
   I have to figure out to use what could be a simple api. I'm sure you've
   been through this experience before where you have these various objects
   and you're trying to figure out what they represent (the connection to the
   server? the information to create a connection? a request session?).
   2. We've tried to avoid the Impl naming convention. In general the rule
   has been if there is only going to be one implementation you don't need an
   interface. If there will be multiple, distinguish it from the others. The
   other clients follow this pattern: Producer, KafkaProducer, MockProducer;
   Consumer, KafkaConsumer, MockConsumer.
   3. We generally don't use setters or getters as a naming convention. I
   personally think mutating the setting in place seems kind of like late 90s
   Java style. I think it likely has thread-safety issues. i.e. even if it is
   volatile you may not get the value you just set if there is another
   thread... I actually really liked what you described as your original idea
   of having a single parameter object like CreateTopicRequest that holds all
   these parameters and defaults. This lets you evolve the api with all the
   various combinations of arguments without overloading insanity. After doing
   literally tens of thousands of remote APIs at LinkedIn we eventually
   converged on a rule, which is ultimately every remote api needs a single
   argument object you can add to over time and it must be batched. Which
   brings me to my next point...
   4. I agree batch apis are annoying but I suspect we'll end up adding
   one. Doing 1000 requests for 1000 operations if creating or deleting will
   be bad, right? This won't be the common case, but when you do it it will be
   a deal-breaker problem. I don't think we should try to fix this one behind
   the scenes.
   5. Are we going to do CompletableFuture (which requires java 8) or
   normal Future? Normal Future is utterly useless for most things other than
   just calling wait. If we can evolve in place from Future to
   CompletableFuture that is fantastic (we could do it for the producer too!).
   My belief was that this was binary incompatible but I actually don't know
   (obviously it's source compatible).

-Jay

On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe  wrote:

> Hi all,
>
> I made some major revisions to the proposal on the wiki, so please check
> it out.
>
> The new API is based on Ismael's suggestion of grouping related APIs.
> There is only one layer of grouping.  I think that it's actually pretty
> intuitive.  It's also based on the idea of using Futures, which several
> people commented that they'd like to see.
>
> Here's a simple example:
>
>  > AdminClient client = new AdminClientImpl(myConfig);
>  > try {
>  >   client.topics().create("foo", 3, (short) 2, false).get();
>  >   Collection topicNames = client.topics().list(false).get();
>  >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
>  >   Collection nodes = client.nodes().list().get();
>  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ", "));
>  > } finally {
>  >   client.close();
>  > }
>
> The good thing is, there is no Try, no 'get' prefixes, no messing with
> batch APIs.  If there is an error, then Future#get() throws an
> ExecutionException which wraps the relevant exception in the standard
> Java way.
>
> Here's a slightly 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-11 Thread Dong Lin
If we require use non-batching API for purge, client needs to send one
PurgeRequest per partition and broker needs to have one delayed purgatory
event per partition. This is inefficient if the broker has 10s of
partitions that user wants to purge. This is a general problem for all APIs
(e.g. offsetsForTimes) which are typically used for operation on multiple
partitions at a time.


On Fri, Feb 10, 2017 at 5:05 PM, Dong Lin  wrote:

> Hi Jun,
>
> Currently KIP-107 uses this API:
>
> Future> 
> purgeDataBefore(Map Long> offsetForPartition)
>
> Are you suggesting that we should provide this:
>
> Future purgeDataBefore(TopicPartition, Long)
>
> I think the second solution works and the resulting implementation of
> KIP-107 will be simpler. The only concern is that this will be a bit
> inconvenience to use. User will need to have a "for" loop in their own code
> to call purgeDataBefore for every partition, get results for every
> partition, and aggregate results.
>
> KIP-117 currently only provides topic management operation and it is
> reasonable to call create(..) once for every topic because user generally
> creates one topic a time. But operations such as purgeDataBefore(..) seems
> different in the sense that user generally wants to purge data of all
> partitions of a topic.
>
> I can change change KIP-107 to use the second API if we decide to avoid
> batch API in AdminClient in the long term with the awareness that user
> needs to do a bit extra work. Maybe we can minimize this extra work by
> providing a utility class to combine multiple Future objects into one
> Future() object.
>
> Thanks,
> Dong
>
>
>
> On Fri, Feb 10, 2017 at 4:18 PM, Jun Rao  wrote:
>
>> Hi, Dong,
>>
>> For KIP-107, the purgeDataBefore() api will eventually be added to the
>> AdminClient too, right? It would be useful to make the apis consistent.
>> Currently, in KIP-107, we do batching in purgeDataBefore(). In Colin's
>> current proposal, there is no batching.
>>
>> Thanks,
>>
>> Jun
>>
>> On Thu, Feb 9, 2017 at 10:54 AM, Dong Lin  wrote:
>>
>> > Thanks for the explanation. This makes sense.
>> >
>> > Best,
>> > Dong
>> >
>> > On Thu, Feb 9, 2017 at 10:51 AM, Colin McCabe 
>> wrote:
>> >
>> > > On Wed, Feb 8, 2017, at 19:02, Dong Lin wrote:
>> > > > I am not aware of any semantics that will be caused by sharing
>> > > > NetworkClient between producer/consumer and AdminClient. But I agree
>> > that
>> > > > there is currently no good way to share such an internal class
>> between
>> > > > them. And yes, goal is to reduce number of connections. For example,
>> > say
>> > > > we
>> > > > want to enable auto data purge based on committed offset using
>> > > > AdminClient.purgeDataBefore(...) in a stream processing
>> application,
>> > > then
>> > > > in addition to producer or consumer, we will now have AdminClient in
>> > > > every
>> > > > job. It means that the the number of connection between server and
>> > client
>> > > > will double.
>> > > >
>> > > > I have another comment on the KIP. Is AdminClient API supposed to be
>> > > > thread
>> > > > safe?
>> > >
>> > > Yes.  The wiki page states: "The client will be multi-threaded;
>> multiple
>> > > threads will be able to safely make calls using the same AdminClient
>> > > object."
>> > >
>> > > > If so, should we mark private variables such as clientTimeoutMs to
>> > > > be @volatile? Would it be a concern if two threads call
>> > > > TopicsContext.setServerTimeout(...) concurrently to use different
>> > > timeout
>> > > > for their own use-case?
>> > >
>> > > The context objects are extremely lightweight and are not intended to
>> be
>> > > shared between multiple threads.  So each thread would just do
>> > > client.topics().setClientTimeout(...).create(), and operate on its
>> own
>> > > TopicsContext.  I will add that to the wiki.
>> > >
>> > > best,
>> > > Colin
>> > >
>> > > >
>> > > > Thanks,
>> > > > Dong
>> > > >
>> > > > On Wed, Feb 8, 2017 at 6:50 PM, Jason Gustafson > >
>> > > > wrote:
>> > > >
>> > > > > I'm not too sure sharing NetworkClient is a good idea. The
>> consumer
>> > > and the
>> > > > > producer both have request semantics which would be more
>> difficult to
>> > > > > reason about if the connections are shared with another client.
>> Also,
>> > > the
>> > > > > NetworkClient is an internal class which is not really meant for
>> > > users. Do
>> > > > > we really want to open that up? Is the only benefit saving the
>> number
>> > > of
>> > > > > connections? Seems not worth it in my opinion.
>> > > > >
>> > > > > -Jason
>> > > > >
>> > > > > On Wed, Feb 8, 2017 at 6:43 PM, Dong Lin 
>> > wrote:
>> > > > >
>> > > > > > BTW, the idea to share NetworkClient is suggested by Radai and I
>> > like
>> > > > > this
>> > > > > > idea.
>> > > > > >
>> > > > > > On Wed, Feb 8, 2017 at 6:39 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-10 Thread Dong Lin
Hi Jun,

Currently KIP-107 uses this API:

Future>
purgeDataBefore(Map offsetForPartition)

Are you suggesting that we should provide this:

Future purgeDataBefore(TopicPartition, Long)

I think the second solution works and the resulting implementation of
KIP-107 will be simpler. The only concern is that this will be a bit
inconvenience to use. User will need to have a "for" loop in their own code
to call purgeDataBefore for every partition, get results for every
partition, and aggregate results.

KIP-117 currently only provides topic management operation and it is
reasonable to call create(..) once for every topic because user generally
creates one topic a time. But operations such as purgeDataBefore(..) seems
different in the sense that user generally wants to purge data of all
partitions of a topic.

I can change change KIP-107 to use the second API if we decide to avoid
batch API in AdminClient in the long term with the awareness that user
needs to do a bit extra work. Maybe we can minimize this extra work by
providing a utility class to combine multiple Future objects into one
Future() object.

Thanks,
Dong



On Fri, Feb 10, 2017 at 4:18 PM, Jun Rao  wrote:

> Hi, Dong,
>
> For KIP-107, the purgeDataBefore() api will eventually be added to the
> AdminClient too, right? It would be useful to make the apis consistent.
> Currently, in KIP-107, we do batching in purgeDataBefore(). In Colin's
> current proposal, there is no batching.
>
> Thanks,
>
> Jun
>
> On Thu, Feb 9, 2017 at 10:54 AM, Dong Lin  wrote:
>
> > Thanks for the explanation. This makes sense.
> >
> > Best,
> > Dong
> >
> > On Thu, Feb 9, 2017 at 10:51 AM, Colin McCabe 
> wrote:
> >
> > > On Wed, Feb 8, 2017, at 19:02, Dong Lin wrote:
> > > > I am not aware of any semantics that will be caused by sharing
> > > > NetworkClient between producer/consumer and AdminClient. But I agree
> > that
> > > > there is currently no good way to share such an internal class
> between
> > > > them. And yes, goal is to reduce number of connections. For example,
> > say
> > > > we
> > > > want to enable auto data purge based on committed offset using
> > > > AdminClient.purgeDataBefore(...) in a stream processing application,
> > > then
> > > > in addition to producer or consumer, we will now have AdminClient in
> > > > every
> > > > job. It means that the the number of connection between server and
> > client
> > > > will double.
> > > >
> > > > I have another comment on the KIP. Is AdminClient API supposed to be
> > > > thread
> > > > safe?
> > >
> > > Yes.  The wiki page states: "The client will be multi-threaded;
> multiple
> > > threads will be able to safely make calls using the same AdminClient
> > > object."
> > >
> > > > If so, should we mark private variables such as clientTimeoutMs to
> > > > be @volatile? Would it be a concern if two threads call
> > > > TopicsContext.setServerTimeout(...) concurrently to use different
> > > timeout
> > > > for their own use-case?
> > >
> > > The context objects are extremely lightweight and are not intended to
> be
> > > shared between multiple threads.  So each thread would just do
> > > client.topics().setClientTimeout(...).create(), and operate on its own
> > > TopicsContext.  I will add that to the wiki.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Wed, Feb 8, 2017 at 6:50 PM, Jason Gustafson 
> > > > wrote:
> > > >
> > > > > I'm not too sure sharing NetworkClient is a good idea. The consumer
> > > and the
> > > > > producer both have request semantics which would be more difficult
> to
> > > > > reason about if the connections are shared with another client.
> Also,
> > > the
> > > > > NetworkClient is an internal class which is not really meant for
> > > users. Do
> > > > > we really want to open that up? Is the only benefit saving the
> number
> > > of
> > > > > connections? Seems not worth it in my opinion.
> > > > >
> > > > > -Jason
> > > > >
> > > > > On Wed, Feb 8, 2017 at 6:43 PM, Dong Lin 
> > wrote:
> > > > >
> > > > > > BTW, the idea to share NetworkClient is suggested by Radai and I
> > like
> > > > > this
> > > > > > idea.
> > > > > >
> > > > > > On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin 
> > > wrote:
> > > > > >
> > > > > > > Hey Colin,
> > > > > > >
> > > > > > > Thanks for updating the KIP. I have two followup questions:
> > > > > > >
> > > > > > > - It seems that setCreationConfig(...) is a bit redundant given
> > > that
> > > > > most
> > > > > > > arguments (e.g. topic name, partition num) are already passed
> to
> > > > > > > TopicsContext.create(...) when user creates topic. Should we
> pass
> > > > > > > the creationConfig as a parameter to TopicsContext.create(..)?
> > > > > > >
> > > > > > > - I am wondering if we should also specify the constructor of
> the
> > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-10 Thread Jun Rao
Hi, Dong,

For KIP-107, the purgeDataBefore() api will eventually be added to the
AdminClient too, right? It would be useful to make the apis consistent.
Currently, in KIP-107, we do batching in purgeDataBefore(). In Colin's
current proposal, there is no batching.

Thanks,

Jun

On Thu, Feb 9, 2017 at 10:54 AM, Dong Lin  wrote:

> Thanks for the explanation. This makes sense.
>
> Best,
> Dong
>
> On Thu, Feb 9, 2017 at 10:51 AM, Colin McCabe  wrote:
>
> > On Wed, Feb 8, 2017, at 19:02, Dong Lin wrote:
> > > I am not aware of any semantics that will be caused by sharing
> > > NetworkClient between producer/consumer and AdminClient. But I agree
> that
> > > there is currently no good way to share such an internal class between
> > > them. And yes, goal is to reduce number of connections. For example,
> say
> > > we
> > > want to enable auto data purge based on committed offset using
> > > AdminClient.purgeDataBefore(...) in a stream processing application,
> > then
> > > in addition to producer or consumer, we will now have AdminClient in
> > > every
> > > job. It means that the the number of connection between server and
> client
> > > will double.
> > >
> > > I have another comment on the KIP. Is AdminClient API supposed to be
> > > thread
> > > safe?
> >
> > Yes.  The wiki page states: "The client will be multi-threaded; multiple
> > threads will be able to safely make calls using the same AdminClient
> > object."
> >
> > > If so, should we mark private variables such as clientTimeoutMs to
> > > be @volatile? Would it be a concern if two threads call
> > > TopicsContext.setServerTimeout(...) concurrently to use different
> > timeout
> > > for their own use-case?
> >
> > The context objects are extremely lightweight and are not intended to be
> > shared between multiple threads.  So each thread would just do
> > client.topics().setClientTimeout(...).create(), and operate on its own
> > TopicsContext.  I will add that to the wiki.
> >
> > best,
> > Colin
> >
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Wed, Feb 8, 2017 at 6:50 PM, Jason Gustafson 
> > > wrote:
> > >
> > > > I'm not too sure sharing NetworkClient is a good idea. The consumer
> > and the
> > > > producer both have request semantics which would be more difficult to
> > > > reason about if the connections are shared with another client. Also,
> > the
> > > > NetworkClient is an internal class which is not really meant for
> > users. Do
> > > > we really want to open that up? Is the only benefit saving the number
> > of
> > > > connections? Seems not worth it in my opinion.
> > > >
> > > > -Jason
> > > >
> > > > On Wed, Feb 8, 2017 at 6:43 PM, Dong Lin 
> wrote:
> > > >
> > > > > BTW, the idea to share NetworkClient is suggested by Radai and I
> like
> > > > this
> > > > > idea.
> > > > >
> > > > > On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin 
> > wrote:
> > > > >
> > > > > > Hey Colin,
> > > > > >
> > > > > > Thanks for updating the KIP. I have two followup questions:
> > > > > >
> > > > > > - It seems that setCreationConfig(...) is a bit redundant given
> > that
> > > > most
> > > > > > arguments (e.g. topic name, partition num) are already passed to
> > > > > > TopicsContext.create(...) when user creates topic. Should we pass
> > > > > > the creationConfig as a parameter to TopicsContext.create(..)?
> > > > > >
> > > > > > - I am wondering if we should also specify the constructor of the
> > > > > > AdminClient in the KIP. Previously we agreed that AdminClient
> > should
> > > > have
> > > > > > its own thread to poll NetworkClient to send/receive messages.
> > Should
> > > > we
> > > > > > also allow AdminClient to use an existing NetworkClient that is
> > > > provided
> > > > > to
> > > > > > the constructor? This would allow AdminClient to share
> > NetworkClient
> > > > with
> > > > > > producer or consumer in order to reduce the total number of open
> > > > sockets
> > > > > on
> > > > > > both client and server.
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > > > On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe  >
> > > > wrote:
> > > > > >
> > > > > >> Hi all,
> > > > > >>
> > > > > >> I made some major revisions to the proposal on the wiki, so
> please
> > > > check
> > > > > >> it out.
> > > > > >>
> > > > > >> The new API is based on Ismael's suggestion of grouping related
> > APIs.
> > > > > >> There is only one layer of grouping.  I think that it's actually
> > > > pretty
> > > > > >> intuitive.  It's also based on the idea of using Futures, which
> > > > several
> > > > > >> people commented that they'd like to see.
> > > > > >>
> > > > > >> Here's a simple example:
> > > > > >>
> > > > > >>  > AdminClient client = new AdminClientImpl(myConfig);
> > > > > >>  > try {
> > > > > >>  >   client.topics().create("foo", 3, (short) 2, false).get();
> > > > > >>  >   Collection topicNames =
> 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-09 Thread Dong Lin
Thanks for the explanation. This makes sense.

Best,
Dong

On Thu, Feb 9, 2017 at 10:51 AM, Colin McCabe  wrote:

> On Wed, Feb 8, 2017, at 19:02, Dong Lin wrote:
> > I am not aware of any semantics that will be caused by sharing
> > NetworkClient between producer/consumer and AdminClient. But I agree that
> > there is currently no good way to share such an internal class between
> > them. And yes, goal is to reduce number of connections. For example, say
> > we
> > want to enable auto data purge based on committed offset using
> > AdminClient.purgeDataBefore(...) in a stream processing application,
> then
> > in addition to producer or consumer, we will now have AdminClient in
> > every
> > job. It means that the the number of connection between server and client
> > will double.
> >
> > I have another comment on the KIP. Is AdminClient API supposed to be
> > thread
> > safe?
>
> Yes.  The wiki page states: "The client will be multi-threaded; multiple
> threads will be able to safely make calls using the same AdminClient
> object."
>
> > If so, should we mark private variables such as clientTimeoutMs to
> > be @volatile? Would it be a concern if two threads call
> > TopicsContext.setServerTimeout(...) concurrently to use different
> timeout
> > for their own use-case?
>
> The context objects are extremely lightweight and are not intended to be
> shared between multiple threads.  So each thread would just do
> client.topics().setClientTimeout(...).create(), and operate on its own
> TopicsContext.  I will add that to the wiki.
>
> best,
> Colin
>
> >
> > Thanks,
> > Dong
> >
> > On Wed, Feb 8, 2017 at 6:50 PM, Jason Gustafson 
> > wrote:
> >
> > > I'm not too sure sharing NetworkClient is a good idea. The consumer
> and the
> > > producer both have request semantics which would be more difficult to
> > > reason about if the connections are shared with another client. Also,
> the
> > > NetworkClient is an internal class which is not really meant for
> users. Do
> > > we really want to open that up? Is the only benefit saving the number
> of
> > > connections? Seems not worth it in my opinion.
> > >
> > > -Jason
> > >
> > > On Wed, Feb 8, 2017 at 6:43 PM, Dong Lin  wrote:
> > >
> > > > BTW, the idea to share NetworkClient is suggested by Radai and I like
> > > this
> > > > idea.
> > > >
> > > > On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin 
> wrote:
> > > >
> > > > > Hey Colin,
> > > > >
> > > > > Thanks for updating the KIP. I have two followup questions:
> > > > >
> > > > > - It seems that setCreationConfig(...) is a bit redundant given
> that
> > > most
> > > > > arguments (e.g. topic name, partition num) are already passed to
> > > > > TopicsContext.create(...) when user creates topic. Should we pass
> > > > > the creationConfig as a parameter to TopicsContext.create(..)?
> > > > >
> > > > > - I am wondering if we should also specify the constructor of the
> > > > > AdminClient in the KIP. Previously we agreed that AdminClient
> should
> > > have
> > > > > its own thread to poll NetworkClient to send/receive messages.
> Should
> > > we
> > > > > also allow AdminClient to use an existing NetworkClient that is
> > > provided
> > > > to
> > > > > the constructor? This would allow AdminClient to share
> NetworkClient
> > > with
> > > > > producer or consumer in order to reduce the total number of open
> > > sockets
> > > > on
> > > > > both client and server.
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > > On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe 
> > > wrote:
> > > > >
> > > > >> Hi all,
> > > > >>
> > > > >> I made some major revisions to the proposal on the wiki, so please
> > > check
> > > > >> it out.
> > > > >>
> > > > >> The new API is based on Ismael's suggestion of grouping related
> APIs.
> > > > >> There is only one layer of grouping.  I think that it's actually
> > > pretty
> > > > >> intuitive.  It's also based on the idea of using Futures, which
> > > several
> > > > >> people commented that they'd like to see.
> > > > >>
> > > > >> Here's a simple example:
> > > > >>
> > > > >>  > AdminClient client = new AdminClientImpl(myConfig);
> > > > >>  > try {
> > > > >>  >   client.topics().create("foo", 3, (short) 2, false).get();
> > > > >>  >   Collection topicNames = client.topics().list(false).
> > > get();
> > > > >>  >   log.info("Found topics: {}", Utils.mkString(topicNames, ",
> "));
> > > > >>  >   Collection nodes = client.nodes().list().get();
> > > > >>  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ",
> > > "));
> > > > >>  > } finally {
> > > > >>  >   client.close();
> > > > >>  > }
> > > > >>
> > > > >> The good thing is, there is no Try, no 'get' prefixes, no messing
> with
> > > > >> batch APIs.  If there is an error, then Future#get() throws an
> > > > >> ExecutionException which wraps the relevant exception in the
> standard
> > > > >> Java way.
> > > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-09 Thread Colin McCabe
On Wed, Feb 8, 2017, at 19:02, Dong Lin wrote:
> I am not aware of any semantics that will be caused by sharing
> NetworkClient between producer/consumer and AdminClient. But I agree that
> there is currently no good way to share such an internal class between
> them. And yes, goal is to reduce number of connections. For example, say
> we
> want to enable auto data purge based on committed offset using
> AdminClient.purgeDataBefore(...) in a stream processing application, then
> in addition to producer or consumer, we will now have AdminClient in
> every
> job. It means that the the number of connection between server and client
> will double.
> 
> I have another comment on the KIP. Is AdminClient API supposed to be
> thread
> safe? 

Yes.  The wiki page states: "The client will be multi-threaded; multiple
threads will be able to safely make calls using the same AdminClient
object."

> If so, should we mark private variables such as clientTimeoutMs to
> be @volatile? Would it be a concern if two threads call
> TopicsContext.setServerTimeout(...) concurrently to use different timeout
> for their own use-case?

The context objects are extremely lightweight and are not intended to be
shared between multiple threads.  So each thread would just do
client.topics().setClientTimeout(...).create(), and operate on its own
TopicsContext.  I will add that to the wiki.

best,
Colin

> 
> Thanks,
> Dong
> 
> On Wed, Feb 8, 2017 at 6:50 PM, Jason Gustafson 
> wrote:
> 
> > I'm not too sure sharing NetworkClient is a good idea. The consumer and the
> > producer both have request semantics which would be more difficult to
> > reason about if the connections are shared with another client. Also, the
> > NetworkClient is an internal class which is not really meant for users. Do
> > we really want to open that up? Is the only benefit saving the number of
> > connections? Seems not worth it in my opinion.
> >
> > -Jason
> >
> > On Wed, Feb 8, 2017 at 6:43 PM, Dong Lin  wrote:
> >
> > > BTW, the idea to share NetworkClient is suggested by Radai and I like
> > this
> > > idea.
> > >
> > > On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin  wrote:
> > >
> > > > Hey Colin,
> > > >
> > > > Thanks for updating the KIP. I have two followup questions:
> > > >
> > > > - It seems that setCreationConfig(...) is a bit redundant given that
> > most
> > > > arguments (e.g. topic name, partition num) are already passed to
> > > > TopicsContext.create(...) when user creates topic. Should we pass
> > > > the creationConfig as a parameter to TopicsContext.create(..)?
> > > >
> > > > - I am wondering if we should also specify the constructor of the
> > > > AdminClient in the KIP. Previously we agreed that AdminClient should
> > have
> > > > its own thread to poll NetworkClient to send/receive messages. Should
> > we
> > > > also allow AdminClient to use an existing NetworkClient that is
> > provided
> > > to
> > > > the constructor? This would allow AdminClient to share NetworkClient
> > with
> > > > producer or consumer in order to reduce the total number of open
> > sockets
> > > on
> > > > both client and server.
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > > > On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe 
> > wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> I made some major revisions to the proposal on the wiki, so please
> > check
> > > >> it out.
> > > >>
> > > >> The new API is based on Ismael's suggestion of grouping related APIs.
> > > >> There is only one layer of grouping.  I think that it's actually
> > pretty
> > > >> intuitive.  It's also based on the idea of using Futures, which
> > several
> > > >> people commented that they'd like to see.
> > > >>
> > > >> Here's a simple example:
> > > >>
> > > >>  > AdminClient client = new AdminClientImpl(myConfig);
> > > >>  > try {
> > > >>  >   client.topics().create("foo", 3, (short) 2, false).get();
> > > >>  >   Collection topicNames = client.topics().list(false).
> > get();
> > > >>  >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
> > > >>  >   Collection nodes = client.nodes().list().get();
> > > >>  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ",
> > "));
> > > >>  > } finally {
> > > >>  >   client.close();
> > > >>  > }
> > > >>
> > > >> The good thing is, there is no Try, no 'get' prefixes, no messing with
> > > >> batch APIs.  If there is an error, then Future#get() throws an
> > > >> ExecutionException which wraps the relevant exception in the standard
> > > >> Java way.
> > > >>
> > > >> Here's a slightly less simple example:
> > > >>
> > > >> > AdminClient client = new AdminClientImpl(myConfig);
> > > >> > try {
> > > >> >   List futures = new LinkedList<>();
> > > >> >   for (String topicName: myNewTopicNames) {
> > > >> > creations.add(client.topics().
> > > >> > setClientTimeout(3).setCreationConfig(myTopicConfig).
> > > >> >   

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-09 Thread Colin McCabe
On Wed, Feb 8, 2017, at 18:39, Dong Lin wrote:
> Hey Colin,
> 
> Thanks for updating the KIP. I have two followup questions:
> 
> - It seems that setCreationConfig(...) is a bit redundant given that most
> arguments (e.g. topic name, partition num) are already passed to
> TopicsContext.create(...) when user creates topic. Should we pass
> the creationConfig as a parameter to TopicsContext.create(..)?

That makes sense.  I'll add it as a parameter to create.

> 
> - I am wondering if we should also specify the constructor of the
> AdminClient in the KIP. Previously we agreed that AdminClient should have
> its own thread to poll NetworkClient to send/receive messages. Should we
> also allow AdminClient to use an existing NetworkClient that is provided
> to
> the constructor? This would allow AdminClient to share NetworkClient with
> producer or consumer in order to reduce the total number of open sockets
> on
> both client and server.

Hmm.  NetworkClient isn't thread safe.  There is no safe way for the
AdminClient thread to share a NetworkClient instance with any other
thread.  I also think library users shouldn't be accessing NetworkClient
since it's part of the implementation, not the API.

It would be interesting to think about extending NetworkClient so that
it could support being shared between consumer, producer, and
adminclient.  You would probably also need to support having multiple
requests in flight to a particular node, which it can't do right now. 
We should probably do that in a separate KIP, though.

best,
Colin

> 
> Thanks,
> Dong
> 
> On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe  wrote:
> 
> > Hi all,
> >
> > I made some major revisions to the proposal on the wiki, so please check
> > it out.
> >
> > The new API is based on Ismael's suggestion of grouping related APIs.
> > There is only one layer of grouping.  I think that it's actually pretty
> > intuitive.  It's also based on the idea of using Futures, which several
> > people commented that they'd like to see.
> >
> > Here's a simple example:
> >
> >  > AdminClient client = new AdminClientImpl(myConfig);
> >  > try {
> >  >   client.topics().create("foo", 3, (short) 2, false).get();
> >  >   Collection topicNames = client.topics().list(false).get();
> >  >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
> >  >   Collection nodes = client.nodes().list().get();
> >  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ", "));
> >  > } finally {
> >  >   client.close();
> >  > }
> >
> > The good thing is, there is no Try, no 'get' prefixes, no messing with
> > batch APIs.  If there is an error, then Future#get() throws an
> > ExecutionException which wraps the relevant exception in the standard
> > Java way.
> >
> > Here's a slightly less simple example:
> >
> > > AdminClient client = new AdminClientImpl(myConfig);
> > > try {
> > >   List futures = new LinkedList<>();
> > >   for (String topicName: myNewTopicNames) {
> > > creations.add(client.topics().
> > > setClientTimeout(3).setCreationConfig(myTopicConfig).
> > >   create(topicName, 3, (short) 2, false));
> > >   }
> > >   Futures.waitForAll(futures);
> > > } finally {
> > >   client.close();
> > > }
> >
> > I went with Futures because I feel like ought to have some option for
> > doing async.  It's a style of programming that has become a lot more
> > popular with the rise of Node.js, Twisted python, etc. etc.  Also, as
> > Ismael commented, Java 8 CompletableFuture is going to make Java's
> > support for fluent async programming a lot stronger by allowing call
> > chaining and much more.
> >
> > If we are going to support async, the simplest thing is just to make
> > everything return a future and let people call get() if they want to run
> > synchronously.  Having a mix of async and sync APIs is just going to be
> > confusing and redundant.
> >
> > I think we should try to avoid creating single functions that start
> > multiple requests if we can.  It makes things much uglier.  It means
> > that you have to have some kind of request class that wraps up the
> > request the user is trying to create, so that you can handle an array of
> > those requests.  The return value has to be something like Map > Try> to represent which nodes failed and succeeded.  This is the
> > kind of stuff that, in my opinion, makes people scratch their heads.
> >
> > If we need to, we can still get some of the efficiency benefits of batch
> > APIs by waiting for a millisecond or two before sending out a topic
> > create() request to see if other create() requests arrive.  If so, we
> > can coalesce them.  It might be better to figure out if this is an
> > actual performance issue before implementing it, though.
> >
> > I think it would be good to get something out there, annotate it as
> > @Unstable, and get feedback from people building against trunk and using
> > it.  We have removed or changed @Unstable APIs in 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Dong Lin
I am not aware of any semantics that will be caused by sharing
NetworkClient between producer/consumer and AdminClient. But I agree that
there is currently no good way to share such an internal class between
them. And yes, goal is to reduce number of connections. For example, say we
want to enable auto data purge based on committed offset using
AdminClient.purgeDataBefore(...) in a stream processing application, then
in addition to producer or consumer, we will now have AdminClient in every
job. It means that the the number of connection between server and client
will double.

I have another comment on the KIP. Is AdminClient API supposed to be thread
safe? If so, should we mark private variables such as clientTimeoutMs to be
@volatile? Would it be a concern if two threads call
TopicsContext.setServerTimeout(...) concurrently to use different timeout
for their own use-case?

Thanks,
Dong

On Wed, Feb 8, 2017 at 6:50 PM, Jason Gustafson  wrote:

> I'm not too sure sharing NetworkClient is a good idea. The consumer and the
> producer both have request semantics which would be more difficult to
> reason about if the connections are shared with another client. Also, the
> NetworkClient is an internal class which is not really meant for users. Do
> we really want to open that up? Is the only benefit saving the number of
> connections? Seems not worth it in my opinion.
>
> -Jason
>
> On Wed, Feb 8, 2017 at 6:43 PM, Dong Lin  wrote:
>
> > BTW, the idea to share NetworkClient is suggested by Radai and I like
> this
> > idea.
> >
> > On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin  wrote:
> >
> > > Hey Colin,
> > >
> > > Thanks for updating the KIP. I have two followup questions:
> > >
> > > - It seems that setCreationConfig(...) is a bit redundant given that
> most
> > > arguments (e.g. topic name, partition num) are already passed to
> > > TopicsContext.create(...) when user creates topic. Should we pass
> > > the creationConfig as a parameter to TopicsContext.create(..)?
> > >
> > > - I am wondering if we should also specify the constructor of the
> > > AdminClient in the KIP. Previously we agreed that AdminClient should
> have
> > > its own thread to poll NetworkClient to send/receive messages. Should
> we
> > > also allow AdminClient to use an existing NetworkClient that is
> provided
> > to
> > > the constructor? This would allow AdminClient to share NetworkClient
> with
> > > producer or consumer in order to reduce the total number of open
> sockets
> > on
> > > both client and server.
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe 
> wrote:
> > >
> > >> Hi all,
> > >>
> > >> I made some major revisions to the proposal on the wiki, so please
> check
> > >> it out.
> > >>
> > >> The new API is based on Ismael's suggestion of grouping related APIs.
> > >> There is only one layer of grouping.  I think that it's actually
> pretty
> > >> intuitive.  It's also based on the idea of using Futures, which
> several
> > >> people commented that they'd like to see.
> > >>
> > >> Here's a simple example:
> > >>
> > >>  > AdminClient client = new AdminClientImpl(myConfig);
> > >>  > try {
> > >>  >   client.topics().create("foo", 3, (short) 2, false).get();
> > >>  >   Collection topicNames = client.topics().list(false).
> get();
> > >>  >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
> > >>  >   Collection nodes = client.nodes().list().get();
> > >>  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ",
> "));
> > >>  > } finally {
> > >>  >   client.close();
> > >>  > }
> > >>
> > >> The good thing is, there is no Try, no 'get' prefixes, no messing with
> > >> batch APIs.  If there is an error, then Future#get() throws an
> > >> ExecutionException which wraps the relevant exception in the standard
> > >> Java way.
> > >>
> > >> Here's a slightly less simple example:
> > >>
> > >> > AdminClient client = new AdminClientImpl(myConfig);
> > >> > try {
> > >> >   List futures = new LinkedList<>();
> > >> >   for (String topicName: myNewTopicNames) {
> > >> > creations.add(client.topics().
> > >> > setClientTimeout(3).setCreationConfig(myTopicConfig).
> > >> >   create(topicName, 3, (short) 2, false));
> > >> >   }
> > >> >   Futures.waitForAll(futures);
> > >> > } finally {
> > >> >   client.close();
> > >> > }
> > >>
> > >> I went with Futures because I feel like ought to have some option for
> > >> doing async.  It's a style of programming that has become a lot more
> > >> popular with the rise of Node.js, Twisted python, etc. etc.  Also, as
> > >> Ismael commented, Java 8 CompletableFuture is going to make Java's
> > >> support for fluent async programming a lot stronger by allowing call
> > >> chaining and much more.
> > >>
> > >> If we are going to support async, the simplest thing is just to make
> > >> everything return a future and let 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Jason Gustafson
I'm not too sure sharing NetworkClient is a good idea. The consumer and the
producer both have request semantics which would be more difficult to
reason about if the connections are shared with another client. Also, the
NetworkClient is an internal class which is not really meant for users. Do
we really want to open that up? Is the only benefit saving the number of
connections? Seems not worth it in my opinion.

-Jason

On Wed, Feb 8, 2017 at 6:43 PM, Dong Lin  wrote:

> BTW, the idea to share NetworkClient is suggested by Radai and I like this
> idea.
>
> On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin  wrote:
>
> > Hey Colin,
> >
> > Thanks for updating the KIP. I have two followup questions:
> >
> > - It seems that setCreationConfig(...) is a bit redundant given that most
> > arguments (e.g. topic name, partition num) are already passed to
> > TopicsContext.create(...) when user creates topic. Should we pass
> > the creationConfig as a parameter to TopicsContext.create(..)?
> >
> > - I am wondering if we should also specify the constructor of the
> > AdminClient in the KIP. Previously we agreed that AdminClient should have
> > its own thread to poll NetworkClient to send/receive messages. Should we
> > also allow AdminClient to use an existing NetworkClient that is provided
> to
> > the constructor? This would allow AdminClient to share NetworkClient with
> > producer or consumer in order to reduce the total number of open sockets
> on
> > both client and server.
> >
> > Thanks,
> > Dong
> >
> > On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe  wrote:
> >
> >> Hi all,
> >>
> >> I made some major revisions to the proposal on the wiki, so please check
> >> it out.
> >>
> >> The new API is based on Ismael's suggestion of grouping related APIs.
> >> There is only one layer of grouping.  I think that it's actually pretty
> >> intuitive.  It's also based on the idea of using Futures, which several
> >> people commented that they'd like to see.
> >>
> >> Here's a simple example:
> >>
> >>  > AdminClient client = new AdminClientImpl(myConfig);
> >>  > try {
> >>  >   client.topics().create("foo", 3, (short) 2, false).get();
> >>  >   Collection topicNames = client.topics().list(false).get();
> >>  >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
> >>  >   Collection nodes = client.nodes().list().get();
> >>  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ", "));
> >>  > } finally {
> >>  >   client.close();
> >>  > }
> >>
> >> The good thing is, there is no Try, no 'get' prefixes, no messing with
> >> batch APIs.  If there is an error, then Future#get() throws an
> >> ExecutionException which wraps the relevant exception in the standard
> >> Java way.
> >>
> >> Here's a slightly less simple example:
> >>
> >> > AdminClient client = new AdminClientImpl(myConfig);
> >> > try {
> >> >   List futures = new LinkedList<>();
> >> >   for (String topicName: myNewTopicNames) {
> >> > creations.add(client.topics().
> >> > setClientTimeout(3).setCreationConfig(myTopicConfig).
> >> >   create(topicName, 3, (short) 2, false));
> >> >   }
> >> >   Futures.waitForAll(futures);
> >> > } finally {
> >> >   client.close();
> >> > }
> >>
> >> I went with Futures because I feel like ought to have some option for
> >> doing async.  It's a style of programming that has become a lot more
> >> popular with the rise of Node.js, Twisted python, etc. etc.  Also, as
> >> Ismael commented, Java 8 CompletableFuture is going to make Java's
> >> support for fluent async programming a lot stronger by allowing call
> >> chaining and much more.
> >>
> >> If we are going to support async, the simplest thing is just to make
> >> everything return a future and let people call get() if they want to run
> >> synchronously.  Having a mix of async and sync APIs is just going to be
> >> confusing and redundant.
> >>
> >> I think we should try to avoid creating single functions that start
> >> multiple requests if we can.  It makes things much uglier.  It means
> >> that you have to have some kind of request class that wraps up the
> >> request the user is trying to create, so that you can handle an array of
> >> those requests.  The return value has to be something like Map >> Try> to represent which nodes failed and succeeded.  This is the
> >> kind of stuff that, in my opinion, makes people scratch their heads.
> >>
> >> If we need to, we can still get some of the efficiency benefits of batch
> >> APIs by waiting for a millisecond or two before sending out a topic
> >> create() request to see if other create() requests arrive.  If so, we
> >> can coalesce them.  It might be better to figure out if this is an
> >> actual performance issue before implementing it, though.
> >>
> >> I think it would be good to get something out there, annotate it as
> >> @Unstable, and get feedback from people building against trunk and using
> >> 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Dong Lin
BTW, the idea to share NetworkClient is suggested by Radai and I like this
idea.

On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin  wrote:

> Hey Colin,
>
> Thanks for updating the KIP. I have two followup questions:
>
> - It seems that setCreationConfig(...) is a bit redundant given that most
> arguments (e.g. topic name, partition num) are already passed to
> TopicsContext.create(...) when user creates topic. Should we pass
> the creationConfig as a parameter to TopicsContext.create(..)?
>
> - I am wondering if we should also specify the constructor of the
> AdminClient in the KIP. Previously we agreed that AdminClient should have
> its own thread to poll NetworkClient to send/receive messages. Should we
> also allow AdminClient to use an existing NetworkClient that is provided to
> the constructor? This would allow AdminClient to share NetworkClient with
> producer or consumer in order to reduce the total number of open sockets on
> both client and server.
>
> Thanks,
> Dong
>
> On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe  wrote:
>
>> Hi all,
>>
>> I made some major revisions to the proposal on the wiki, so please check
>> it out.
>>
>> The new API is based on Ismael's suggestion of grouping related APIs.
>> There is only one layer of grouping.  I think that it's actually pretty
>> intuitive.  It's also based on the idea of using Futures, which several
>> people commented that they'd like to see.
>>
>> Here's a simple example:
>>
>>  > AdminClient client = new AdminClientImpl(myConfig);
>>  > try {
>>  >   client.topics().create("foo", 3, (short) 2, false).get();
>>  >   Collection topicNames = client.topics().list(false).get();
>>  >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
>>  >   Collection nodes = client.nodes().list().get();
>>  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ", "));
>>  > } finally {
>>  >   client.close();
>>  > }
>>
>> The good thing is, there is no Try, no 'get' prefixes, no messing with
>> batch APIs.  If there is an error, then Future#get() throws an
>> ExecutionException which wraps the relevant exception in the standard
>> Java way.
>>
>> Here's a slightly less simple example:
>>
>> > AdminClient client = new AdminClientImpl(myConfig);
>> > try {
>> >   List futures = new LinkedList<>();
>> >   for (String topicName: myNewTopicNames) {
>> > creations.add(client.topics().
>> > setClientTimeout(3).setCreationConfig(myTopicConfig).
>> >   create(topicName, 3, (short) 2, false));
>> >   }
>> >   Futures.waitForAll(futures);
>> > } finally {
>> >   client.close();
>> > }
>>
>> I went with Futures because I feel like ought to have some option for
>> doing async.  It's a style of programming that has become a lot more
>> popular with the rise of Node.js, Twisted python, etc. etc.  Also, as
>> Ismael commented, Java 8 CompletableFuture is going to make Java's
>> support for fluent async programming a lot stronger by allowing call
>> chaining and much more.
>>
>> If we are going to support async, the simplest thing is just to make
>> everything return a future and let people call get() if they want to run
>> synchronously.  Having a mix of async and sync APIs is just going to be
>> confusing and redundant.
>>
>> I think we should try to avoid creating single functions that start
>> multiple requests if we can.  It makes things much uglier.  It means
>> that you have to have some kind of request class that wraps up the
>> request the user is trying to create, so that you can handle an array of
>> those requests.  The return value has to be something like Map> Try> to represent which nodes failed and succeeded.  This is the
>> kind of stuff that, in my opinion, makes people scratch their heads.
>>
>> If we need to, we can still get some of the efficiency benefits of batch
>> APIs by waiting for a millisecond or two before sending out a topic
>> create() request to see if other create() requests arrive.  If so, we
>> can coalesce them.  It might be better to figure out if this is an
>> actual performance issue before implementing it, though.
>>
>> I think it would be good to get something out there, annotate it as
>> @Unstable, and get feedback from people building against trunk and using
>> it.  We have removed or changed @Unstable APIs in streams before, so I
>> don't think we should worry that it will get set in stone prematurely.
>> The AdminClient API should get much less developer use than anything in
>> streams, so changing an unstable API should be much easier.
>>
>> best,
>> Colin
>>
>>
>> On Wed, Feb 8, 2017, at 07:49, Ismael Juma wrote:
>> > Thanks for elaborating Jay. I totally agree that we have to be very
>> > careful
>> > in how we use our complexity budget. Easier said than done when people
>> > don't agree on what is complex and what is simple. :) For example, I
>> > think
>> > batch APIs are a significant source of complexity as you have to do a
>> > bunch

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Dong Lin
Hey Colin,

Thanks for updating the KIP. I have two followup questions:

- It seems that setCreationConfig(...) is a bit redundant given that most
arguments (e.g. topic name, partition num) are already passed to
TopicsContext.create(...) when user creates topic. Should we pass
the creationConfig as a parameter to TopicsContext.create(..)?

- I am wondering if we should also specify the constructor of the
AdminClient in the KIP. Previously we agreed that AdminClient should have
its own thread to poll NetworkClient to send/receive messages. Should we
also allow AdminClient to use an existing NetworkClient that is provided to
the constructor? This would allow AdminClient to share NetworkClient with
producer or consumer in order to reduce the total number of open sockets on
both client and server.

Thanks,
Dong

On Wed, Feb 8, 2017 at 5:00 PM, Colin McCabe  wrote:

> Hi all,
>
> I made some major revisions to the proposal on the wiki, so please check
> it out.
>
> The new API is based on Ismael's suggestion of grouping related APIs.
> There is only one layer of grouping.  I think that it's actually pretty
> intuitive.  It's also based on the idea of using Futures, which several
> people commented that they'd like to see.
>
> Here's a simple example:
>
>  > AdminClient client = new AdminClientImpl(myConfig);
>  > try {
>  >   client.topics().create("foo", 3, (short) 2, false).get();
>  >   Collection topicNames = client.topics().list(false).get();
>  >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
>  >   Collection nodes = client.nodes().list().get();
>  >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ", "));
>  > } finally {
>  >   client.close();
>  > }
>
> The good thing is, there is no Try, no 'get' prefixes, no messing with
> batch APIs.  If there is an error, then Future#get() throws an
> ExecutionException which wraps the relevant exception in the standard
> Java way.
>
> Here's a slightly less simple example:
>
> > AdminClient client = new AdminClientImpl(myConfig);
> > try {
> >   List futures = new LinkedList<>();
> >   for (String topicName: myNewTopicNames) {
> > creations.add(client.topics().
> > setClientTimeout(3).setCreationConfig(myTopicConfig).
> >   create(topicName, 3, (short) 2, false));
> >   }
> >   Futures.waitForAll(futures);
> > } finally {
> >   client.close();
> > }
>
> I went with Futures because I feel like ought to have some option for
> doing async.  It's a style of programming that has become a lot more
> popular with the rise of Node.js, Twisted python, etc. etc.  Also, as
> Ismael commented, Java 8 CompletableFuture is going to make Java's
> support for fluent async programming a lot stronger by allowing call
> chaining and much more.
>
> If we are going to support async, the simplest thing is just to make
> everything return a future and let people call get() if they want to run
> synchronously.  Having a mix of async and sync APIs is just going to be
> confusing and redundant.
>
> I think we should try to avoid creating single functions that start
> multiple requests if we can.  It makes things much uglier.  It means
> that you have to have some kind of request class that wraps up the
> request the user is trying to create, so that you can handle an array of
> those requests.  The return value has to be something like Map Try> to represent which nodes failed and succeeded.  This is the
> kind of stuff that, in my opinion, makes people scratch their heads.
>
> If we need to, we can still get some of the efficiency benefits of batch
> APIs by waiting for a millisecond or two before sending out a topic
> create() request to see if other create() requests arrive.  If so, we
> can coalesce them.  It might be better to figure out if this is an
> actual performance issue before implementing it, though.
>
> I think it would be good to get something out there, annotate it as
> @Unstable, and get feedback from people building against trunk and using
> it.  We have removed or changed @Unstable APIs in streams before, so I
> don't think we should worry that it will get set in stone prematurely.
> The AdminClient API should get much less developer use than anything in
> streams, so changing an unstable API should be much easier.
>
> best,
> Colin
>
>
> On Wed, Feb 8, 2017, at 07:49, Ismael Juma wrote:
> > Thanks for elaborating Jay. I totally agree that we have to be very
> > careful
> > in how we use our complexity budget. Easier said than done when people
> > don't agree on what is complex and what is simple. :) For example, I
> > think
> > batch APIs are a significant source of complexity as you have to do a
> > bunch
> > of ceremony to group things before sending the request and error handling
> > becomes more complex due to partial failures (things like `Try` or other
> > mechanisms that serve a similar role are then needed).
> >
> > Maybe a way forward is to write API usage examples to 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Colin McCabe
Hi all,

I made some major revisions to the proposal on the wiki, so please check
it out.

The new API is based on Ismael's suggestion of grouping related APIs. 
There is only one layer of grouping.  I think that it's actually pretty
intuitive.  It's also based on the idea of using Futures, which several
people commented that they'd like to see.

Here's a simple example:

 > AdminClient client = new AdminClientImpl(myConfig);
 > try {
 >   client.topics().create("foo", 3, (short) 2, false).get();
 >   Collection topicNames = client.topics().list(false).get();
 >   log.info("Found topics: {}", Utils.mkString(topicNames, ", "));
 >   Collection nodes = client.nodes().list().get();
 >   log.info("Found cluster nodes: {}", Utils.mkString(nodes, ", "));
 > } finally {
 >   client.close();
 > }

The good thing is, there is no Try, no 'get' prefixes, no messing with
batch APIs.  If there is an error, then Future#get() throws an
ExecutionException which wraps the relevant exception in the standard
Java way.

Here's a slightly less simple example:

> AdminClient client = new AdminClientImpl(myConfig);
> try {
>   List futures = new LinkedList<>();
>   for (String topicName: myNewTopicNames) {
> creations.add(client.topics().
> setClientTimeout(3).setCreationConfig(myTopicConfig).
>   create(topicName, 3, (short) 2, false));
>   }
>   Futures.waitForAll(futures);
> } finally {
>   client.close();
> }

I went with Futures because I feel like ought to have some option for
doing async.  It's a style of programming that has become a lot more
popular with the rise of Node.js, Twisted python, etc. etc.  Also, as
Ismael commented, Java 8 CompletableFuture is going to make Java's
support for fluent async programming a lot stronger by allowing call
chaining and much more.

If we are going to support async, the simplest thing is just to make
everything return a future and let people call get() if they want to run
synchronously.  Having a mix of async and sync APIs is just going to be
confusing and redundant.

I think we should try to avoid creating single functions that start
multiple requests if we can.  It makes things much uglier.  It means
that you have to have some kind of request class that wraps up the
request the user is trying to create, so that you can handle an array of
those requests.  The return value has to be something like Map to represent which nodes failed and succeeded.  This is the
kind of stuff that, in my opinion, makes people scratch their heads.

If we need to, we can still get some of the efficiency benefits of batch
APIs by waiting for a millisecond or two before sending out a topic
create() request to see if other create() requests arrive.  If so, we
can coalesce them.  It might be better to figure out if this is an
actual performance issue before implementing it, though.

I think it would be good to get something out there, annotate it as
@Unstable, and get feedback from people building against trunk and using
it.  We have removed or changed @Unstable APIs in streams before, so I
don't think we should worry that it will get set in stone prematurely. 
The AdminClient API should get much less developer use than anything in
streams, so changing an unstable API should be much easier.

best,
Colin


On Wed, Feb 8, 2017, at 07:49, Ismael Juma wrote:
> Thanks for elaborating Jay. I totally agree that we have to be very
> careful
> in how we use our complexity budget. Easier said than done when people
> don't agree on what is complex and what is simple. :) For example, I
> think
> batch APIs are a significant source of complexity as you have to do a
> bunch
> of ceremony to group things before sending the request and error handling
> becomes more complex due to partial failures (things like `Try` or other
> mechanisms that serve a similar role are then needed).
> 
> Maybe a way forward is to write API usage examples to help validate that
> the suggested API is indeed easy to use.
> 
> Ismael
> 
> On Wed, Feb 8, 2017 at 4:40 AM, Jay Kreps  wrote:
> 
> > Totally agree on CompletableFuture. Also agree with some of the rough edges
> > on the Consumer.
> >
> > I don't have much of a leg to stand on with the splitting vs not splitting
> > thing, really hard to argue one or the other is better. I guess the one
> > observation in watching us try to make good public apis over the years is I
> > am kind of in favor of a particular kind of simple. In particular I think
> > since the bar is sooo high in support and docs and the community of users
> > so broad in the range of their capabilities, it makes it so there is a lot
> > of value in dead simple interfaces that don't have a lot of conceptual
> > weight, don't introduce a lot of new classes or concepts or general
> > patterns that must be understood to use them correctly. So things like
> > nesting, or the Try class, or async apis, or even just a complex set of
> > classes representing arguments or 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Ismael Juma
Thanks for elaborating Jay. I totally agree that we have to be very careful
in how we use our complexity budget. Easier said than done when people
don't agree on what is complex and what is simple. :) For example, I think
batch APIs are a significant source of complexity as you have to do a bunch
of ceremony to group things before sending the request and error handling
becomes more complex due to partial failures (things like `Try` or other
mechanisms that serve a similar role are then needed).

Maybe a way forward is to write API usage examples to help validate that
the suggested API is indeed easy to use.

Ismael

On Wed, Feb 8, 2017 at 4:40 AM, Jay Kreps  wrote:

> Totally agree on CompletableFuture. Also agree with some of the rough edges
> on the Consumer.
>
> I don't have much of a leg to stand on with the splitting vs not splitting
> thing, really hard to argue one or the other is better. I guess the one
> observation in watching us try to make good public apis over the years is I
> am kind of in favor of a particular kind of simple. In particular I think
> since the bar is sooo high in support and docs and the community of users
> so broad in the range of their capabilities, it makes it so there is a lot
> of value in dead simple interfaces that don't have a lot of conceptual
> weight, don't introduce a lot of new classes or concepts or general
> patterns that must be understood to use them correctly. So things like
> nesting, or the Try class, or async apis, or even just a complex set of
> classes representing arguments or return values kind of all stack
> conceptual burdens on the user to figure out correct usage. So like, for
> example, the Try class is very elegant and represents a whole generalized
> class of possibly completed actions, but the flip side is maybe I'm just a
> working guy who needs to list his kafka topics but doesn't know Rust, take
> pity on me! :-)
>
> Nit picking aside, super excited to see us progress on this.
>
> -Jay
>
> On Tue, Feb 7, 2017 at 3:46 PM Ismael Juma  wrote:
>
> > Hi Jay,
> >
> > Thanks for the feedback. Comments inline.
> >
> > On Tue, Feb 7, 2017 at 8:18 PM, Jay Kreps  wrote:
> > >
> > >- I think it would be good to not use "get" as the prefix for things
> > >making remote calls. We've tried to avoid the java getter convention
> > >entirely (see code style guide), but for remote calls in particular
> it
> > > kind
> > >of blurs the line between field access and remote RPC in a way that
> > > leads
> > >people to trouble. What about, e.g., fetchAllGroups() vs
> > getAllGroups().
> > >
> >
> > Agreed that we should avoid the `get` prefix for remote calls. There are
> a
> > few possible prefixes for the read operations: list, fetch, describe.
> >
> >
> > >- I think futures and callbacks are a bit of a pain to use. I'd
> second
> > >Becket's comment: let's ensure there a common use case motivating
> > these
> > >that wouldn't be just as easily satisfied with batch operations
> (which
> > > we
> > >seem to have at least for some things). In terms of flexibility
> > > Callbacks >
> > >Futures > Batch Ops but I think in terms of usability it is the
> exact
> > >opposite so let's make sure we have worked out how the API will be
> > used
> > >before deciding. In particular I think java Futures are often an
> > >uncomfortable half-way point since calling get() and blocking the
> > > thread is
> > >often not what you want for chaining sequences of operations in a
> > truly
> > >async way, so 99% of people just use the future as a way to batch
> > calls.
> > >
> >
> > We should definitely figure out how the APIs are going to be used before
> > deciding. I agree that callbacks are definitely painful and there's
> little
> > reason to expose them in a modern API unless it's meant to be very low
> > level. When it comes to Futures, I think it's important to distinguish
> what
> > is available in Java 7 and below versus what is available from Java 8
> > onwards. CompletableFuture makes it much easier to compose/chain
> operations
> > (in a similar vein to java.util.Stream, our own Streams API, etc.) and it
> > gives you the ability to register callbacks if you really want to
> (avoiding
> > the somewhat odd situation in the Producer where we return a Future _and_
> > allow you to pass a callback).
> >
> >
> > >- Personally I don't think splitting the admin methods up actually
> > makes
> > >things more usable. It just makes you have to dig through our
> > > hierarchy. I
> > >think a flat class with a bunch of operations (like the consumer
> api)
> > is
> > >probably the easiest for people to grok and find things on. I am
> kind
> > > of a
> > >dumb PHP programmer at heart, though.
> > >
> >
> > I am not sure it's fair to compare the AdminClient with the Consumer. The
> > former has APIs for a bunch of unrelated APIs (topics, ACLs, 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Jay Kreps
Totally agree on CompletableFuture. Also agree with some of the rough edges
on the Consumer.

I don't have much of a leg to stand on with the splitting vs not splitting
thing, really hard to argue one or the other is better. I guess the one
observation in watching us try to make good public apis over the years is I
am kind of in favor of a particular kind of simple. In particular I think
since the bar is sooo high in support and docs and the community of users
so broad in the range of their capabilities, it makes it so there is a lot
of value in dead simple interfaces that don't have a lot of conceptual
weight, don't introduce a lot of new classes or concepts or general
patterns that must be understood to use them correctly. So things like
nesting, or the Try class, or async apis, or even just a complex set of
classes representing arguments or return values kind of all stack
conceptual burdens on the user to figure out correct usage. So like, for
example, the Try class is very elegant and represents a whole generalized
class of possibly completed actions, but the flip side is maybe I'm just a
working guy who needs to list his kafka topics but doesn't know Rust, take
pity on me! :-)

Nit picking aside, super excited to see us progress on this.

-Jay

On Tue, Feb 7, 2017 at 3:46 PM Ismael Juma  wrote:

> Hi Jay,
>
> Thanks for the feedback. Comments inline.
>
> On Tue, Feb 7, 2017 at 8:18 PM, Jay Kreps  wrote:
> >
> >- I think it would be good to not use "get" as the prefix for things
> >making remote calls. We've tried to avoid the java getter convention
> >entirely (see code style guide), but for remote calls in particular it
> > kind
> >of blurs the line between field access and remote RPC in a way that
> > leads
> >people to trouble. What about, e.g., fetchAllGroups() vs
> getAllGroups().
> >
>
> Agreed that we should avoid the `get` prefix for remote calls. There are a
> few possible prefixes for the read operations: list, fetch, describe.
>
>
> >- I think futures and callbacks are a bit of a pain to use. I'd second
> >Becket's comment: let's ensure there a common use case motivating
> these
> >that wouldn't be just as easily satisfied with batch operations (which
> > we
> >seem to have at least for some things). In terms of flexibility
> > Callbacks >
> >Futures > Batch Ops but I think in terms of usability it is the exact
> >opposite so let's make sure we have worked out how the API will be
> used
> >before deciding. In particular I think java Futures are often an
> >uncomfortable half-way point since calling get() and blocking the
> > thread is
> >often not what you want for chaining sequences of operations in a
> truly
> >async way, so 99% of people just use the future as a way to batch
> calls.
> >
>
> We should definitely figure out how the APIs are going to be used before
> deciding. I agree that callbacks are definitely painful and there's little
> reason to expose them in a modern API unless it's meant to be very low
> level. When it comes to Futures, I think it's important to distinguish what
> is available in Java 7 and below versus what is available from Java 8
> onwards. CompletableFuture makes it much easier to compose/chain operations
> (in a similar vein to java.util.Stream, our own Streams API, etc.) and it
> gives you the ability to register callbacks if you really want to (avoiding
> the somewhat odd situation in the Producer where we return a Future _and_
> allow you to pass a callback).
>
>
> >- Personally I don't think splitting the admin methods up actually
> makes
> >things more usable. It just makes you have to dig through our
> > hierarchy. I
> >think a flat class with a bunch of operations (like the consumer api)
> is
> >probably the easiest for people to grok and find things on. I am kind
> > of a
> >dumb PHP programmer at heart, though.
> >
>
> I am not sure it's fair to compare the AdminClient with the Consumer. The
> former has APIs for a bunch of unrelated APIs (topics, ACLs, configs,
> consumer groups, delegation tokens, preferred leader election, partition
> reassignment, etc.) where the latter is pretty specialised. For each of the
> resources, you may have 3-4 operations, it will get confusing fast. Also,
> do you really think an API that has one level of grouping will mean that
> users have to "dig through our hierarchy"? Or are you concerned that once
> we go in that direction, there is a danger of making the hierarchy more
> complicated?
>
> Finally, I am not sure I would use the consumer as an example of something
> that is easy to grok. :) The fact that methods behave pretty differently
> (some are blocking while others only have an effect after poll) with no
> indication from the type signature or naming convention makes it harder,
> not easier, to understand.
>
> Ismael
>


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread William Gray
+1 for placing the AdminClient in a separate package.

I have several dozen applications that use kafka-clients.jar to
produce/consume, and those applications don't have a need for the
AdminClient. And, it's probably better not to have the AdminClient methods
available in those applications from a management standpoint.

Right now I only have one application that uses the internal scala
AdminClient (from Java), and that's for ops and monitoring purposes. I'll
refactor that app to use the Java native client once it's available. I use
AdminClient.describeGroup(groupId), which doesn't look like it's making the
cut, so I'll look to extend the client or send a pull request.

Bill


On Tue, Feb 7, 2017 at 6:27 PM, Gwen Shapira  wrote:

> Ismael raised the concern (offline) that even if we mark the API as
> unstable, people will use it right away because it is so valuable. So
> we'll have trouble changing later if we prefer a different API.
>
> I agree, but since I still prefer to experiment a bit "in the wild", I
> suggested placing the AdminClient in its own Maven package (separate
> from kafka-clients). This will allow users to upgrade their
> AdminClient dependency when they want to, without worrying about being
> stuck with an older producer/consumer version at the same time. It
> should give us better ability to experiment since users can stay on
> the older APIs as long as they want.
>
> IMO, we can leave the packages separate forever and avoid pains of
> merging them. Does anyone see downsides to separate packages?
>
> On Tue, Feb 7, 2017 at 4:17 PM, Gwen Shapira  wrote:
> >>>- Personally I don't think splitting the admin methods up actually
> makes
> >>>things more usable. It just makes you have to dig through our
> >>> hierarchy. I
> >>>think a flat class with a bunch of operations (like the consumer
> api) is
> >>>probably the easiest for people to grok and find things on. I am
> kind
> >>> of a
> >>>dumb PHP programmer at heart, though.
> >>>
> >>
> >> I am not sure it's fair to compare the AdminClient with the Consumer.
> The
> >> former has APIs for a bunch of unrelated APIs (topics, ACLs, configs,
> >> consumer groups, delegation tokens, preferred leader election, partition
> >> reassignment, etc.) where the latter is pretty specialised. For each of
> the
> >> resources, you may have 3-4 operations, it will get confusing fast.
> Also,
> >> do you really think an API that has one level of grouping will mean that
> >> users have to "dig through our hierarchy"? Or are you concerned that
> once
> >> we go in that direction, there is a danger of making the hierarchy more
> >> complicated?
> >>
> >> Finally, I am not sure I would use the consumer as an example of
> something
> >> that is easy to grok. :) The fact that methods behave pretty differently
> >> (some are blocking while others only have an effect after poll) with no
> >> indication from the type signature or naming convention makes it harder,
> >> not easier, to understand.
> >
> >
> > I also wouldn't consider the consumer a shining example of usability.
> >
> > Streams went in the "split stuff" direction. At the upper level, you
> > can either create a Stream or a Table. Then you have relevant methods
> > under each. They even have a bit more hierarchy: GroupedStream under
> > stream where aggregates live. I thought it works rather well once you
> > figure out the rules.
> >
> > But I think there is a class of engineers who learn how to use a
> > client by creating an instance in IntelliJ and then browse through the
> > list of methods available and see what looks right. Hierarchies throw
> > us off a bit off. I remember browsing KStream and freaking out because
> > I can't find "aggregate" method. So we need to make sure this thing
> > makes sense to new users (we can find a few to test on?).
> >
> > Since moving stuff around a hierarchy is fairly easy, why not try one
> > way, leave things as "unstable" for one release and gather feedback?
> > We usually get usability complaints fairly fast.
> >
> >>
> >> Ismael
> >
> >
> >
> > --
> > Gwen Shapira
> > Product Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter | blog
>
>
>
> --
> Gwen Shapira
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Gwen Shapira
Ismael raised the concern (offline) that even if we mark the API as
unstable, people will use it right away because it is so valuable. So
we'll have trouble changing later if we prefer a different API.

I agree, but since I still prefer to experiment a bit "in the wild", I
suggested placing the AdminClient in its own Maven package (separate
from kafka-clients). This will allow users to upgrade their
AdminClient dependency when they want to, without worrying about being
stuck with an older producer/consumer version at the same time. It
should give us better ability to experiment since users can stay on
the older APIs as long as they want.

IMO, we can leave the packages separate forever and avoid pains of
merging them. Does anyone see downsides to separate packages?

On Tue, Feb 7, 2017 at 4:17 PM, Gwen Shapira  wrote:
>>>- Personally I don't think splitting the admin methods up actually makes
>>>things more usable. It just makes you have to dig through our
>>> hierarchy. I
>>>think a flat class with a bunch of operations (like the consumer api) is
>>>probably the easiest for people to grok and find things on. I am kind
>>> of a
>>>dumb PHP programmer at heart, though.
>>>
>>
>> I am not sure it's fair to compare the AdminClient with the Consumer. The
>> former has APIs for a bunch of unrelated APIs (topics, ACLs, configs,
>> consumer groups, delegation tokens, preferred leader election, partition
>> reassignment, etc.) where the latter is pretty specialised. For each of the
>> resources, you may have 3-4 operations, it will get confusing fast. Also,
>> do you really think an API that has one level of grouping will mean that
>> users have to "dig through our hierarchy"? Or are you concerned that once
>> we go in that direction, there is a danger of making the hierarchy more
>> complicated?
>>
>> Finally, I am not sure I would use the consumer as an example of something
>> that is easy to grok. :) The fact that methods behave pretty differently
>> (some are blocking while others only have an effect after poll) with no
>> indication from the type signature or naming convention makes it harder,
>> not easier, to understand.
>
>
> I also wouldn't consider the consumer a shining example of usability.
>
> Streams went in the "split stuff" direction. At the upper level, you
> can either create a Stream or a Table. Then you have relevant methods
> under each. They even have a bit more hierarchy: GroupedStream under
> stream where aggregates live. I thought it works rather well once you
> figure out the rules.
>
> But I think there is a class of engineers who learn how to use a
> client by creating an instance in IntelliJ and then browse through the
> list of methods available and see what looks right. Hierarchies throw
> us off a bit off. I remember browsing KStream and freaking out because
> I can't find "aggregate" method. So we need to make sure this thing
> makes sense to new users (we can find a few to test on?).
>
> Since moving stuff around a hierarchy is fairly easy, why not try one
> way, leave things as "unstable" for one release and gather feedback?
> We usually get usability complaints fairly fast.
>
>>
>> Ismael
>
>
>
> --
> Gwen Shapira
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog



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


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Gwen Shapira
>>- Personally I don't think splitting the admin methods up actually makes
>>things more usable. It just makes you have to dig through our
>> hierarchy. I
>>think a flat class with a bunch of operations (like the consumer api) is
>>probably the easiest for people to grok and find things on. I am kind
>> of a
>>dumb PHP programmer at heart, though.
>>
>
> I am not sure it's fair to compare the AdminClient with the Consumer. The
> former has APIs for a bunch of unrelated APIs (topics, ACLs, configs,
> consumer groups, delegation tokens, preferred leader election, partition
> reassignment, etc.) where the latter is pretty specialised. For each of the
> resources, you may have 3-4 operations, it will get confusing fast. Also,
> do you really think an API that has one level of grouping will mean that
> users have to "dig through our hierarchy"? Or are you concerned that once
> we go in that direction, there is a danger of making the hierarchy more
> complicated?
>
> Finally, I am not sure I would use the consumer as an example of something
> that is easy to grok. :) The fact that methods behave pretty differently
> (some are blocking while others only have an effect after poll) with no
> indication from the type signature or naming convention makes it harder,
> not easier, to understand.


I also wouldn't consider the consumer a shining example of usability.

Streams went in the "split stuff" direction. At the upper level, you
can either create a Stream or a Table. Then you have relevant methods
under each. They even have a bit more hierarchy: GroupedStream under
stream where aggregates live. I thought it works rather well once you
figure out the rules.

But I think there is a class of engineers who learn how to use a
client by creating an instance in IntelliJ and then browse through the
list of methods available and see what looks right. Hierarchies throw
us off a bit off. I remember browsing KStream and freaking out because
I can't find "aggregate" method. So we need to make sure this thing
makes sense to new users (we can find a few to test on?).

Since moving stuff around a hierarchy is fairly easy, why not try one
way, leave things as "unstable" for one release and gather feedback?
We usually get usability complaints fairly fast.

>
> Ismael



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


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Ismael Juma
Hi Jay,

Thanks for the feedback. Comments inline.

On Tue, Feb 7, 2017 at 8:18 PM, Jay Kreps  wrote:
>
>- I think it would be good to not use "get" as the prefix for things
>making remote calls. We've tried to avoid the java getter convention
>entirely (see code style guide), but for remote calls in particular it
> kind
>of blurs the line between field access and remote RPC in a way that
> leads
>people to trouble. What about, e.g., fetchAllGroups() vs getAllGroups().
>

Agreed that we should avoid the `get` prefix for remote calls. There are a
few possible prefixes for the read operations: list, fetch, describe.


>- I think futures and callbacks are a bit of a pain to use. I'd second
>Becket's comment: let's ensure there a common use case motivating these
>that wouldn't be just as easily satisfied with batch operations (which
> we
>seem to have at least for some things). In terms of flexibility
> Callbacks >
>Futures > Batch Ops but I think in terms of usability it is the exact
>opposite so let's make sure we have worked out how the API will be used
>before deciding. In particular I think java Futures are often an
>uncomfortable half-way point since calling get() and blocking the
> thread is
>often not what you want for chaining sequences of operations in a truly
>async way, so 99% of people just use the future as a way to batch calls.
>

We should definitely figure out how the APIs are going to be used before
deciding. I agree that callbacks are definitely painful and there's little
reason to expose them in a modern API unless it's meant to be very low
level. When it comes to Futures, I think it's important to distinguish what
is available in Java 7 and below versus what is available from Java 8
onwards. CompletableFuture makes it much easier to compose/chain operations
(in a similar vein to java.util.Stream, our own Streams API, etc.) and it
gives you the ability to register callbacks if you really want to (avoiding
the somewhat odd situation in the Producer where we return a Future _and_
allow you to pass a callback).


>- Personally I don't think splitting the admin methods up actually makes
>things more usable. It just makes you have to dig through our
> hierarchy. I
>think a flat class with a bunch of operations (like the consumer api) is
>probably the easiest for people to grok and find things on. I am kind
> of a
>dumb PHP programmer at heart, though.
>

I am not sure it's fair to compare the AdminClient with the Consumer. The
former has APIs for a bunch of unrelated APIs (topics, ACLs, configs,
consumer groups, delegation tokens, preferred leader election, partition
reassignment, etc.) where the latter is pretty specialised. For each of the
resources, you may have 3-4 operations, it will get confusing fast. Also,
do you really think an API that has one level of grouping will mean that
users have to "dig through our hierarchy"? Or are you concerned that once
we go in that direction, there is a danger of making the hierarchy more
complicated?

Finally, I am not sure I would use the consumer as an example of something
that is easy to grok. :) The fact that methods behave pretty differently
(some are blocking while others only have an effect after poll) with no
indication from the type signature or naming convention makes it harder,
not easier, to understand.

Ismael


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Jay Kreps
Hey Colin,

This is great, thought I'd throw out a couple of opinions to the mix, feel
free to ignore:

   - I think it would be good to not use "get" as the prefix for things
   making remote calls. We've tried to avoid the java getter convention
   entirely (see code style guide), but for remote calls in particular it kind
   of blurs the line between field access and remote RPC in a way that leads
   people to trouble. What about, e.g., fetchAllGroups() vs getAllGroups().
   - I think futures and callbacks are a bit of a pain to use. I'd second
   Becket's comment: let's ensure there a common use case motivating these
   that wouldn't be just as easily satisfied with batch operations (which we
   seem to have at least for some things). In terms of flexibility Callbacks >
   Futures > Batch Ops but I think in terms of usability it is the exact
   opposite so let's make sure we have worked out how the API will be used
   before deciding. In particular I think java Futures are often an
   uncomfortable half-way point since calling get() and blocking the thread is
   often not what you want for chaining sequences of operations in a truly
   async way, so 99% of people just use the future as a way to batch calls.
   - Personally I don't think splitting the admin methods up actually makes
   things more usable. It just makes you have to dig through our hierarchy. I
   think a flat class with a bunch of operations (like the consumer api) is
   probably the easiest for people to grok and find things on. I am kind of a
   dumb PHP programmer at heart, though.

-Jay

On Tue, Feb 7, 2017 at 10:19 AM, Colin McCabe  wrote:

> On Tue, Feb 7, 2017, at 08:37, Ismael Juma wrote:
> > Hi all,
> >
> > I think it's good that we have discussed a number of API that would make
> > sense in the AdminClient. Having done that, I think we should now narrow
> > the focus of this KIP to a small set of methods to get us started. Once
> > we
> > have an AdminClient in the codebase, we can propose subsequent KIPs to
> > enrich it. I would suggest the following:
> >
> > 1. Let's focus on topic management operations: describe, create, alter
> > and
> > delete topics.
> > 2. Let's add an @Unstable annotation to the class and specify in the
> > javadoc that the methods are subject to change (if necessary).
> >
> > Thoughts?
>
> +1.
>
> I'm going to reorganize the proposal a bit to use futures and to have
> the grouping by API type proposed earlier.
>
> best,
> Colin
>
> >
> > Ismael
> >
> > On Fri, Feb 3, 2017 at 6:24 PM, Colin McCabe  wrote:
> >
> > > On Thu, Feb 2, 2017, at 21:45, Becket Qin wrote:
> > > > Hi Colin,
> > > >
> > > > Thanks for the KIP. An admin client is probably a must after we block
> > > > direct access to ZK. Some comments and thoughts below:
> > > >
> > > > 1. Do we have a clear scope for the admin client? It might be worth
> > > > thinking about the entire user experience of the admin client.
> Ideally we
> > > > may want to have a single client to do all the administrative work
> > > > instead
> > > > of having multiple ways to do different things. For example, do we
> want
> > > > to
> > > > add topic configurations change API in the admin client? What about
> > > > partition movements and preferred leader election? Those are also
> > > > administrative tasks which seem reasonable to be integrated into the
> > > > admin
> > > > client.
> > >
> > > Thanks for the comments, Becket!
> > >
> > > I agree that topic configuration change should be in the administrative
> > > client.  I have not thought about partition movement or preferred
> leader
> > > election.  It probably makes sense to put them in the client as well,
> > > but we should probably have a longer discussion about those features
> > > later when someone is ready to implement them ;)
> > >
> > > best,
> > > Colin
>


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
On Tue, Feb 7, 2017, at 08:37, Ismael Juma wrote:
> Hi all,
> 
> I think it's good that we have discussed a number of API that would make
> sense in the AdminClient. Having done that, I think we should now narrow
> the focus of this KIP to a small set of methods to get us started. Once
> we
> have an AdminClient in the codebase, we can propose subsequent KIPs to
> enrich it. I would suggest the following:
> 
> 1. Let's focus on topic management operations: describe, create, alter
> and
> delete topics.
> 2. Let's add an @Unstable annotation to the class and specify in the
> javadoc that the methods are subject to change (if necessary).
> 
> Thoughts?

+1.

I'm going to reorganize the proposal a bit to use futures and to have
the grouping by API type proposed earlier.

best,
Colin

> 
> Ismael
> 
> On Fri, Feb 3, 2017 at 6:24 PM, Colin McCabe  wrote:
> 
> > On Thu, Feb 2, 2017, at 21:45, Becket Qin wrote:
> > > Hi Colin,
> > >
> > > Thanks for the KIP. An admin client is probably a must after we block
> > > direct access to ZK. Some comments and thoughts below:
> > >
> > > 1. Do we have a clear scope for the admin client? It might be worth
> > > thinking about the entire user experience of the admin client. Ideally we
> > > may want to have a single client to do all the administrative work
> > > instead
> > > of having multiple ways to do different things. For example, do we want
> > > to
> > > add topic configurations change API in the admin client? What about
> > > partition movements and preferred leader election? Those are also
> > > administrative tasks which seem reasonable to be integrated into the
> > > admin
> > > client.
> >
> > Thanks for the comments, Becket!
> >
> > I agree that topic configuration change should be in the administrative
> > client.  I have not thought about partition movement or preferred leader
> > election.  It probably makes sense to put them in the client as well,
> > but we should probably have a longer discussion about those features
> > later when someone is ready to implement them ;)
> >
> > best,
> > Colin


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
Hi Hu Xi,

On Mon, Feb 6, 2017, at 19:37, Hu Xi wrote:
> Two things I want to confirm. Please advise.
> 
> 
> 1. Seems the KIP only cares about topic management things. Is there any
> plan for this KIP to merge the feature of what `GetOffsetShell` script
> offers? Since a lot of people really want to know/monitor how many
> committed records have been created for a topic.

Yes, I think we should provide the ability to get offsets for a topic in
a follow-on KIP.  We are also planning on adding ACL management and
AlterTopic functionality, in follow-on work.

> 
> 
> 2. Since deleting topic is a totally async process, is there any way for
> me to make sure the topic is deleted successfully after invoking
> deleteTopic once the KIP is implemented?

One way would be to periodically list the topics and wait for it to go
away.

best,
Colin

> 
> 
> Regards,
> 
> 
> -- huxi
> 
> 
> 
> 发件人: radai <radai.rosenbl...@gmail.com>
> 发送时间: 2017年2月7日 10:46
> 收件人: dev@kafka.apache.org
> 主题: Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for
> Kafka admin operations
> 
> even assuming all consumers use kafka for offset storage, would it be
> possible to get this information from a single broker without "reaching
> out" to all brokers in a cluster?
> 
> On Mon, Feb 6, 2017 at 2:05 PM, Jianbin Wei <jianbin@netskope.com>
> wrote:
> 
> > In the specify group information, can we also return information like
> > partition assignment for each member, the lag/offset of each
> > member/partition?  It would be useful for Ops/Admin regarding the health of
> > the consumer group.
> >
> > Regards,
> >
> > -- Jianbin
> >
> > > On Feb 6, 2017, at 13:54, Guozhang Wang <wangg...@gmail.com> wrote:
> > >
> > > Some follow-up on 2) / 3) below.
> > >
> > > On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe <cmcc...@apache.org
> > <mailto:cmcc...@apache.org>> wrote:
> > >
> > >> On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> > >>> Thanks for the proposal Colin. A few comments below:
> > >>
> > >> Thanks for taking a look, Guozhang.
> > >>
> > >>>
> > >>> 1. There are a couple of classes that looks new to me but not defined
> > >>> anywhere. For example: NewTopic (topic name and configs?), TopicInfo
> > (is
> > >>> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> > >>> GroupOverview.
> > >>> Could you provide their class definitions?
> > >>
> > >> Good point.  I will add them in the KIP.
> > >>
> > >> NodeApiVersions is at
> > >> ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
> > >>
> > >>>
> > >>> 2. In Streams, we would like to replace its own `
> > >>> org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> > >>> with
> > >>> this new admin client.  One additional request though, is that for
> > create
> > >>> /
> > >>> delete topics, we'd like to use a different "flag" as BLOCKING, which
> > >>> means
> > >>> the response will not be sent back until the controller has updated its
> > >>> own
> > >>> metadata cache for the topic, and even STRICT_BLOCKING, which means the
> > >>> response will not be sent back until the metadata has been propagated
> > to
> > >>> the whole cluster.
> > >>
> > >> Hmm.  It seems like this would require additional RPCs or changes to
> > >> existing RPCs on the server.  So we should handle this in a follow-on
> > >> KIP, I think.
> > >>
> > >>
> > > I agree for STRICT_BLOCKING, while for BLOCKING, it is already supported
> > as
> > > of today I think, and Streams' KafkaClient is using that mechanism as
> > well.
> > >
> > >
> > >>>
> > >>> 3. I'm wondering what's the usage of "public Map<Node,
> > >>> Try<List>> getAllGroups()", or rather, would it be more
> > >>> useful to get a specific group information given the group id?
> > Otherwise
> > >>> we
> > >>> need to query each one of the coordinator.
> > >>
> > >> That's a good point.  We should have an API that gets information about
> > >> a specific group, by querying only the coordinato

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
On Mon, Feb 6, 2017, at 14:05, Jianbin Wei wrote:
> In the specify group information, can we also return information like
> partition assignment for each member, the lag/offset of each
> member/partition?  It would be useful for Ops/Admin regarding the health
> of the consumer group.

This information isn't in the groups RPCs yet, so it would need a
separate KIP to implement.  Let's talk about this more once we have the
basic API in.

best,
Colin


> 
> Regards,
> 
> -- Jianbin
> 
> > On Feb 6, 2017, at 13:54, Guozhang Wang  wrote:
> > 
> > Some follow-up on 2) / 3) below.
> > 
> > On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe  > > wrote:
> > 
> >> On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> >>> Thanks for the proposal Colin. A few comments below:
> >> 
> >> Thanks for taking a look, Guozhang.
> >> 
> >>> 
> >>> 1. There are a couple of classes that looks new to me but not defined
> >>> anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is
> >>> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> >>> GroupOverview.
> >>> Could you provide their class definitions?
> >> 
> >> Good point.  I will add them in the KIP.
> >> 
> >> NodeApiVersions is at
> >> ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
> >> 
> >>> 
> >>> 2. In Streams, we would like to replace its own `
> >>> org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> >>> with
> >>> this new admin client.  One additional request though, is that for create
> >>> /
> >>> delete topics, we'd like to use a different "flag" as BLOCKING, which
> >>> means
> >>> the response will not be sent back until the controller has updated its
> >>> own
> >>> metadata cache for the topic, and even STRICT_BLOCKING, which means the
> >>> response will not be sent back until the metadata has been propagated to
> >>> the whole cluster.
> >> 
> >> Hmm.  It seems like this would require additional RPCs or changes to
> >> existing RPCs on the server.  So we should handle this in a follow-on
> >> KIP, I think.
> >> 
> >> 
> > I agree for STRICT_BLOCKING, while for BLOCKING, it is already supported as
> > of today I think, and Streams' KafkaClient is using that mechanism as well.
> > 
> > 
> >>> 
> >>> 3. I'm wondering what's the usage of "public Map >>> Try> getAllGroups()", or rather, would it be more
> >>> useful to get a specific group information given the group id? Otherwise
> >>> we
> >>> need to query each one of the coordinator.
> >> 
> >> That's a good point.  We should have an API that gets information about
> >> a specific group, by querying only the coordinator for that group.  By
> >> the way, what specific group information should we expose, besides name
> >> and protocolType?
> >> 
> >> 
> > I think these can all be returned?
> > 
> > (groupID, protocolType, generationID, state, members: [memberID,
> > clientHost], leaderID (nullable) )
> > 
> > 
> >>> 
> >>> 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
> >>> with a KafkaAdminClient impl, this at least allows easier mocks for unit
> >>> testing.
> >> 
> >> Yeah, I agree.  Hopefully that will also make it a little clearer what
> >> the boundary is between the internal functions and classes and the
> >> public API.  I'll update the KIP accordingly.
> >> 
> >> thanks,
> >> Colin
> >> 
> >>> 
> >>> Guozhang
> >>> 
> >>> 
> >>> 
> >>> On Fri, Feb 3, 2017 at 10:40 AM, Colin McCabe 
> >> wrote:
> >>> 
>  On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> > Hi Colin,
> > 
> > Thanks for the KIP, great to make progress on this. I have some
> >> initial
> > comments, will post more later:
> > 
> > 1. We have KafkaProducer that implements the Producer interface and
> > KafkaConsumer that implements the Consumer interface. Maybe we could
> >> have
> > KafkaAdminClient that implements the AdminClient interface? Or maybe
> >> just
> > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't
> >> think
> > we
> > should worry about a name clash with the internal AdminClient
> >> written in
> > Scala. That will go away soon enough and choosing a good name for the
> > public class is what matters.
>  
>  Hi Ismael,
>  
>  Thanks for taking a look.
>  
>  I guess my thought process was that users might find it confusing if
> >> the
>  public API and the old private API had the same name.  "What do you
>  mean, I have to upgrade to release X to get AdminClient, I have it
> >> right
>  here?"  I do have a slight preference for the shorter name, though, so
>  if this isn't a worry, we can change it to AdminClient.
>  
> > 
> > 2. We should include the proposed package name in the KIP
> > (probably org.apache.kafka.clients.admin?).
>  
>  Good idea.  I will add the package name to the 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
On Mon, Feb 6, 2017, at 13:54, Guozhang Wang wrote:
> Some follow-up on 2) / 3) below.
> 
> On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe  wrote:
> 
> > On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> > > Thanks for the proposal Colin. A few comments below:
> >
> > Thanks for taking a look, Guozhang.
> >
> > >
> > > 1. There are a couple of classes that looks new to me but not defined
> > > anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is
> > > this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> > > GroupOverview.
> > > Could you provide their class definitions?
> >
> > Good point.  I will add them in the KIP.
> >
> > NodeApiVersions is at
> > ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
> >
> > >
> > > 2. In Streams, we would like to replace its own `
> > > org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> > > with
> > > this new admin client.  One additional request though, is that for create
> > > /
> > > delete topics, we'd like to use a different "flag" as BLOCKING, which
> > > means
> > > the response will not be sent back until the controller has updated its
> > > own
> > > metadata cache for the topic, and even STRICT_BLOCKING, which means the
> > > response will not be sent back until the metadata has been propagated to
> > > the whole cluster.
> >
> > Hmm.  It seems like this would require additional RPCs or changes to
> > existing RPCs on the server.  So we should handle this in a follow-on
> > KIP, I think.
> >
> >
> I agree for STRICT_BLOCKING, while for BLOCKING, it is already supported
> as
> of today I think, and Streams' KafkaClient is using that mechanism as
> well.

Hmm.  I think this is covered by the existing API proposal, which has a
NONBLOCKING flag to indicate setting a server-side timeout of 0, and
uses the AdminClient timeout otherwise.

> 
> 
> > >
> > > 3. I'm wondering what's the usage of "public Map > > Try> getAllGroups()", or rather, would it be more
> > > useful to get a specific group information given the group id? Otherwise
> > > we
> > > need to query each one of the coordinator.
> >
> > That's a good point.  We should have an API that gets information about
> > a specific group, by querying only the coordinator for that group.  By
> > the way, what specific group information should we expose, besides name
> > and protocolType?
> >
> >
> I think these can all be returned?
> 
> (groupID, protocolType, generationID, state, members: [memberID,
> clientHost], leaderID (nullable) )

generationID doesn't seem to be exposed in the DescribeGroupsResponse. 
The others are there, though.

best,
Colin

> 
> 
> > >
> > > 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
> > > with a KafkaAdminClient impl, this at least allows easier mocks for unit
> > > testing.
> >
> > Yeah, I agree.  Hopefully that will also make it a little clearer what
> > the boundary is between the internal functions and classes and the
> > public API.  I'll update the KIP accordingly.
> >
> > thanks,
> > Colin
> >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Fri, Feb 3, 2017 at 10:40 AM, Colin McCabe 
> > wrote:
> > >
> > > > On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> > > > > Hi Colin,
> > > > >
> > > > > Thanks for the KIP, great to make progress on this. I have some
> > initial
> > > > > comments, will post more later:
> > > > >
> > > > > 1. We have KafkaProducer that implements the Producer interface and
> > > > > KafkaConsumer that implements the Consumer interface. Maybe we could
> > have
> > > > > KafkaAdminClient that implements the AdminClient interface? Or maybe
> > just
> > > > > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't
> > think
> > > > > we
> > > > > should worry about a name clash with the internal AdminClient
> > written in
> > > > > Scala. That will go away soon enough and choosing a good name for the
> > > > > public class is what matters.
> > > >
> > > > Hi Ismael,
> > > >
> > > > Thanks for taking a look.
> > > >
> > > > I guess my thought process was that users might find it confusing if
> > the
> > > > public API and the old private API had the same name.  "What do you
> > > > mean, I have to upgrade to release X to get AdminClient, I have it
> > right
> > > > here?"  I do have a slight preference for the shorter name, though, so
> > > > if this isn't a worry, we can change it to AdminClient.
> > > >
> > > > >
> > > > > 2. We should include the proposed package name in the KIP
> > > > > (probably org.apache.kafka.clients.admin?).
> > > >
> > > > Good idea.  I will add the package name to the KIP.
> > > >
> > > > >
> > > > > 3. It would be good to list the supported configs.
> > > >
> > > > OK
> > > >
> > > > >
> > > > > 4. KIP-107, which passed the vote, specifies the introduction of a
> > method
> > > > > to AdminClient with the following signature. We should figure out
> > how it

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Sriram Subramanian
+1


> On Feb 7, 2017, at 9:17 AM, radai  wrote:
> 
> +1.
> 
> under ismael's "facet" approach we could always start with
> AdminClient.topics() and then pile on more of them later.
> 
>> On Tue, Feb 7, 2017 at 8:57 AM, Grant Henke  wrote:
>> 
>> +1 I think its important to focus this KIP discussion on the "patterns" we
>> would like to see in the client and a few key methods in order to make
>> progress and then iterate from there.
>> 
>> I think we should let Colin drive the APIs he thinks are important since he
>> is volunteering to do the work. And then others can propose and add APIs
>> from there.
>> 
>>> On Tue, Feb 7, 2017 at 10:37 AM, Ismael Juma  wrote:
>>> 
>>> Hi all,
>>> 
>>> I think it's good that we have discussed a number of API that would make
>>> sense in the AdminClient. Having done that, I think we should now narrow
>>> the focus of this KIP to a small set of methods to get us started. Once
>> we
>>> have an AdminClient in the codebase, we can propose subsequent KIPs to
>>> enrich it. I would suggest the following:
>>> 
>>> 1. Let's focus on topic management operations: describe, create, alter
>> and
>>> delete topics.
>>> 2. Let's add an @Unstable annotation to the class and specify in the
>>> javadoc that the methods are subject to change (if necessary).
>>> 
>>> Thoughts?
>>> 
>>> Ismael
>>> 
 On Fri, Feb 3, 2017 at 6:24 PM, Colin McCabe  wrote:
 
> On Thu, Feb 2, 2017, at 21:45, Becket Qin wrote:
> Hi Colin,
> 
> Thanks for the KIP. An admin client is probably a must after we block
> direct access to ZK. Some comments and thoughts below:
> 
> 1. Do we have a clear scope for the admin client? It might be worth
> thinking about the entire user experience of the admin client.
>> Ideally
>>> we
> may want to have a single client to do all the administrative work
> instead
> of having multiple ways to do different things. For example, do we
>> want
> to
> add topic configurations change API in the admin client? What about
> partition movements and preferred leader election? Those are also
> administrative tasks which seem reasonable to be integrated into the
> admin
> client.
 
 Thanks for the comments, Becket!
 
 I agree that topic configuration change should be in the administrative
 client.  I have not thought about partition movement or preferred
>> leader
 election.  It probably makes sense to put them in the client as well,
 but we should probably have a longer discussion about those features
 later when someone is ready to implement them ;)
 
 best,
 Colin
>> 
>> 
>> 
>> --
>> Grant Henke
>> Software Engineer | Cloudera
>> gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>> 


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread radai
+1.

under ismael's "facet" approach we could always start with
AdminClient.topics() and then pile on more of them later.

On Tue, Feb 7, 2017 at 8:57 AM, Grant Henke  wrote:

> +1 I think its important to focus this KIP discussion on the "patterns" we
> would like to see in the client and a few key methods in order to make
> progress and then iterate from there.
>
> I think we should let Colin drive the APIs he thinks are important since he
> is volunteering to do the work. And then others can propose and add APIs
> from there.
>
> On Tue, Feb 7, 2017 at 10:37 AM, Ismael Juma  wrote:
>
> > Hi all,
> >
> > I think it's good that we have discussed a number of API that would make
> > sense in the AdminClient. Having done that, I think we should now narrow
> > the focus of this KIP to a small set of methods to get us started. Once
> we
> > have an AdminClient in the codebase, we can propose subsequent KIPs to
> > enrich it. I would suggest the following:
> >
> > 1. Let's focus on topic management operations: describe, create, alter
> and
> > delete topics.
> > 2. Let's add an @Unstable annotation to the class and specify in the
> > javadoc that the methods are subject to change (if necessary).
> >
> > Thoughts?
> >
> > Ismael
> >
> > On Fri, Feb 3, 2017 at 6:24 PM, Colin McCabe  wrote:
> >
> > > On Thu, Feb 2, 2017, at 21:45, Becket Qin wrote:
> > > > Hi Colin,
> > > >
> > > > Thanks for the KIP. An admin client is probably a must after we block
> > > > direct access to ZK. Some comments and thoughts below:
> > > >
> > > > 1. Do we have a clear scope for the admin client? It might be worth
> > > > thinking about the entire user experience of the admin client.
> Ideally
> > we
> > > > may want to have a single client to do all the administrative work
> > > > instead
> > > > of having multiple ways to do different things. For example, do we
> want
> > > > to
> > > > add topic configurations change API in the admin client? What about
> > > > partition movements and preferred leader election? Those are also
> > > > administrative tasks which seem reasonable to be integrated into the
> > > > admin
> > > > client.
> > >
> > > Thanks for the comments, Becket!
> > >
> > > I agree that topic configuration change should be in the administrative
> > > client.  I have not thought about partition movement or preferred
> leader
> > > election.  It probably makes sense to put them in the client as well,
> > > but we should probably have a longer discussion about those features
> > > later when someone is ready to implement them ;)
> > >
> > > best,
> > > Colin
> >
>
>
>
> --
> Grant Henke
> Software Engineer | Cloudera
> gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Grant Henke
+1 I think its important to focus this KIP discussion on the "patterns" we
would like to see in the client and a few key methods in order to make
progress and then iterate from there.

I think we should let Colin drive the APIs he thinks are important since he
is volunteering to do the work. And then others can propose and add APIs
from there.

On Tue, Feb 7, 2017 at 10:37 AM, Ismael Juma  wrote:

> Hi all,
>
> I think it's good that we have discussed a number of API that would make
> sense in the AdminClient. Having done that, I think we should now narrow
> the focus of this KIP to a small set of methods to get us started. Once we
> have an AdminClient in the codebase, we can propose subsequent KIPs to
> enrich it. I would suggest the following:
>
> 1. Let's focus on topic management operations: describe, create, alter and
> delete topics.
> 2. Let's add an @Unstable annotation to the class and specify in the
> javadoc that the methods are subject to change (if necessary).
>
> Thoughts?
>
> Ismael
>
> On Fri, Feb 3, 2017 at 6:24 PM, Colin McCabe  wrote:
>
> > On Thu, Feb 2, 2017, at 21:45, Becket Qin wrote:
> > > Hi Colin,
> > >
> > > Thanks for the KIP. An admin client is probably a must after we block
> > > direct access to ZK. Some comments and thoughts below:
> > >
> > > 1. Do we have a clear scope for the admin client? It might be worth
> > > thinking about the entire user experience of the admin client. Ideally
> we
> > > may want to have a single client to do all the administrative work
> > > instead
> > > of having multiple ways to do different things. For example, do we want
> > > to
> > > add topic configurations change API in the admin client? What about
> > > partition movements and preferred leader election? Those are also
> > > administrative tasks which seem reasonable to be integrated into the
> > > admin
> > > client.
> >
> > Thanks for the comments, Becket!
> >
> > I agree that topic configuration change should be in the administrative
> > client.  I have not thought about partition movement or preferred leader
> > election.  It probably makes sense to put them in the client as well,
> > but we should probably have a longer discussion about those features
> > later when someone is ready to implement them ;)
> >
> > best,
> > Colin
>



-- 
Grant Henke
Software Engineer | Cloudera
gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Ismael Juma
Hi all,

I think it's good that we have discussed a number of API that would make
sense in the AdminClient. Having done that, I think we should now narrow
the focus of this KIP to a small set of methods to get us started. Once we
have an AdminClient in the codebase, we can propose subsequent KIPs to
enrich it. I would suggest the following:

1. Let's focus on topic management operations: describe, create, alter and
delete topics.
2. Let's add an @Unstable annotation to the class and specify in the
javadoc that the methods are subject to change (if necessary).

Thoughts?

Ismael

On Fri, Feb 3, 2017 at 6:24 PM, Colin McCabe  wrote:

> On Thu, Feb 2, 2017, at 21:45, Becket Qin wrote:
> > Hi Colin,
> >
> > Thanks for the KIP. An admin client is probably a must after we block
> > direct access to ZK. Some comments and thoughts below:
> >
> > 1. Do we have a clear scope for the admin client? It might be worth
> > thinking about the entire user experience of the admin client. Ideally we
> > may want to have a single client to do all the administrative work
> > instead
> > of having multiple ways to do different things. For example, do we want
> > to
> > add topic configurations change API in the admin client? What about
> > partition movements and preferred leader election? Those are also
> > administrative tasks which seem reasonable to be integrated into the
> > admin
> > client.
>
> Thanks for the comments, Becket!
>
> I agree that topic configuration change should be in the administrative
> client.  I have not thought about partition movement or preferred leader
> election.  It probably makes sense to put them in the client as well,
> but we should probably have a longer discussion about those features
> later when someone is ready to implement them ;)
>
> best,
> Colin


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Jason Gustafson
It's definitely possible to query an individual group's state efficiently
(that's what GroupCoordinator enables). For the group metadata, I wonder if
the client should focus on consumer groups rather than exposing the full
generality of the group protocol and leaving users to interpret? That would
let the API return the information that users expect (subscriptions,
assignments, and offsets). Kafka Connect also uses the group protocol, but
it could extend an abstract class that is shared with the admin client in
order to expose its own utilities. I think the information we store in the
group metadata for Kafka Connect is less useful anyway.

One quick comment on a couple APIs. This seems awkward:

Map<Node, Try<List>> getAllGroups()

The fact that we need to query each broker individually seems like an
implementation detail which is best left hidden in the AdminClient. We'll
still be stuck with this API even if we come up with a more efficient
approach to collecting this information. It also feels painful to use. Same
for getAllBrokerVersions(). Is there a good reason why we couldn't
aggregate the results for the user?

Thanks,
Jason

On Mon, Feb 6, 2017 at 7:37 PM, Hu Xi <huxi...@hotmail.com> wrote:

> Two things I want to confirm. Please advise.
>
>
> 1. Seems the KIP only cares about topic management things. Is there any
> plan for this KIP to merge the feature of what `GetOffsetShell` script
> offers? Since a lot of people really want to know/monitor how many
> committed records have been created for a topic.
>
>
> 2. Since deleting topic is a totally async process, is there any way for
> me to make sure the topic is deleted successfully after invoking
> deleteTopic once the KIP is implemented?
>
>
> Regards,
>
>
> -- huxi
>
>
> 
> 发件人: radai <radai.rosenbl...@gmail.com>
> 发送时间: 2017年2月7日 10:46
> 收件人: dev@kafka.apache.org
> 主题: Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka
> admin operations
>
> even assuming all consumers use kafka for offset storage, would it be
> possible to get this information from a single broker without "reaching
> out" to all brokers in a cluster?
>
> On Mon, Feb 6, 2017 at 2:05 PM, Jianbin Wei <jianbin@netskope.com>
> wrote:
>
> > In the specify group information, can we also return information like
> > partition assignment for each member, the lag/offset of each
> > member/partition?  It would be useful for Ops/Admin regarding the health
> of
> > the consumer group.
> >
> > Regards,
> >
> > -- Jianbin
> >
> > > On Feb 6, 2017, at 13:54, Guozhang Wang <wangg...@gmail.com> wrote:
> > >
> > > Some follow-up on 2) / 3) below.
> > >
> > > On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe <cmcc...@apache.org
> > <mailto:cmcc...@apache.org>> wrote:
> > >
> > >> On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> > >>> Thanks for the proposal Colin. A few comments below:
> > >>
> > >> Thanks for taking a look, Guozhang.
> > >>
> > >>>
> > >>> 1. There are a couple of classes that looks new to me but not defined
> > >>> anywhere. For example: NewTopic (topic name and configs?), TopicInfo
> > (is
> > >>> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> > >>> GroupOverview.
> > >>> Could you provide their class definitions?
> > >>
> > >> Good point.  I will add them in the KIP.
> > >>
> > >> NodeApiVersions is at
> > >> ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
> > >>
> > >>>
> > >>> 2. In Streams, we would like to replace its own `
> > >>> org.apache.kafka.streams.processor.internals.StreamsKafkaClient`
> class
> > >>> with
> > >>> this new admin client.  One additional request though, is that for
> > create
> > >>> /
> > >>> delete topics, we'd like to use a different "flag" as BLOCKING, which
> > >>> means
> > >>> the response will not be sent back until the controller has updated
> its
> > >>> own
> > >>> metadata cache for the topic, and even STRICT_BLOCKING, which means
> the
> > >>> response will not be sent back until the metadata has been propagated
> > to
> > >>> the whole cluster.
> > >>
> > >> Hmm.  It seems like this would require additional RPCs or changes to
> > >> existing RPCs on the server.  So we should h

[DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Hu Xi
Two things I want to confirm. Please advise.


1. Seems the KIP only cares about topic management things. Is there any plan 
for this KIP to merge the feature of what `GetOffsetShell` script offers? Since 
a lot of people really want to know/monitor how many committed records have 
been created for a topic.


2. Since deleting topic is a totally async process, is there any way for me to 
make sure the topic is deleted successfully after invoking deleteTopic once the 
KIP is implemented?


Regards,


-- huxi



发件人: radai <radai.rosenbl...@gmail.com>
发送时间: 2017年2月7日 10:46
收件人: dev@kafka.apache.org
主题: Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka 
admin operations

even assuming all consumers use kafka for offset storage, would it be
possible to get this information from a single broker without "reaching
out" to all brokers in a cluster?

On Mon, Feb 6, 2017 at 2:05 PM, Jianbin Wei <jianbin@netskope.com>
wrote:

> In the specify group information, can we also return information like
> partition assignment for each member, the lag/offset of each
> member/partition?  It would be useful for Ops/Admin regarding the health of
> the consumer group.
>
> Regards,
>
> -- Jianbin
>
> > On Feb 6, 2017, at 13:54, Guozhang Wang <wangg...@gmail.com> wrote:
> >
> > Some follow-up on 2) / 3) below.
> >
> > On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe <cmcc...@apache.org
> <mailto:cmcc...@apache.org>> wrote:
> >
> >> On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> >>> Thanks for the proposal Colin. A few comments below:
> >>
> >> Thanks for taking a look, Guozhang.
> >>
> >>>
> >>> 1. There are a couple of classes that looks new to me but not defined
> >>> anywhere. For example: NewTopic (topic name and configs?), TopicInfo
> (is
> >>> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> >>> GroupOverview.
> >>> Could you provide their class definitions?
> >>
> >> Good point.  I will add them in the KIP.
> >>
> >> NodeApiVersions is at
> >> ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
> >>
> >>>
> >>> 2. In Streams, we would like to replace its own `
> >>> org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> >>> with
> >>> this new admin client.  One additional request though, is that for
> create
> >>> /
> >>> delete topics, we'd like to use a different "flag" as BLOCKING, which
> >>> means
> >>> the response will not be sent back until the controller has updated its
> >>> own
> >>> metadata cache for the topic, and even STRICT_BLOCKING, which means the
> >>> response will not be sent back until the metadata has been propagated
> to
> >>> the whole cluster.
> >>
> >> Hmm.  It seems like this would require additional RPCs or changes to
> >> existing RPCs on the server.  So we should handle this in a follow-on
> >> KIP, I think.
> >>
> >>
> > I agree for STRICT_BLOCKING, while for BLOCKING, it is already supported
> as
> > of today I think, and Streams' KafkaClient is using that mechanism as
> well.
> >
> >
> >>>
> >>> 3. I'm wondering what's the usage of "public Map<Node,
> >>> Try<List>> getAllGroups()", or rather, would it be more
> >>> useful to get a specific group information given the group id?
> Otherwise
> >>> we
> >>> need to query each one of the coordinator.
> >>
> >> That's a good point.  We should have an API that gets information about
> >> a specific group, by querying only the coordinator for that group.  By
> >> the way, what specific group information should we expose, besides name
> >> and protocolType?
> >>
> >>
> > I think these can all be returned?
> >
> > (groupID, protocolType, generationID, state, members: [memberID,
> > clientHost], leaderID (nullable) )
> >
> >
> >>>
> >>> 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
> >>> with a KafkaAdminClient impl, this at least allows easier mocks for
> unit
> >>> testing.
> >>
> >> Yeah, I agree.  Hopefully that will also make it a little clearer what
> >> the boundary is between the internal functions and classes and the
> >> public API.  I'll update the KIP accordingly.
> >>
> >

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread radai
even assuming all consumers use kafka for offset storage, would it be
possible to get this information from a single broker without "reaching
out" to all brokers in a cluster?

On Mon, Feb 6, 2017 at 2:05 PM, Jianbin Wei 
wrote:

> In the specify group information, can we also return information like
> partition assignment for each member, the lag/offset of each
> member/partition?  It would be useful for Ops/Admin regarding the health of
> the consumer group.
>
> Regards,
>
> -- Jianbin
>
> > On Feb 6, 2017, at 13:54, Guozhang Wang  wrote:
> >
> > Some follow-up on 2) / 3) below.
> >
> > On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe  > wrote:
> >
> >> On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> >>> Thanks for the proposal Colin. A few comments below:
> >>
> >> Thanks for taking a look, Guozhang.
> >>
> >>>
> >>> 1. There are a couple of classes that looks new to me but not defined
> >>> anywhere. For example: NewTopic (topic name and configs?), TopicInfo
> (is
> >>> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> >>> GroupOverview.
> >>> Could you provide their class definitions?
> >>
> >> Good point.  I will add them in the KIP.
> >>
> >> NodeApiVersions is at
> >> ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
> >>
> >>>
> >>> 2. In Streams, we would like to replace its own `
> >>> org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> >>> with
> >>> this new admin client.  One additional request though, is that for
> create
> >>> /
> >>> delete topics, we'd like to use a different "flag" as BLOCKING, which
> >>> means
> >>> the response will not be sent back until the controller has updated its
> >>> own
> >>> metadata cache for the topic, and even STRICT_BLOCKING, which means the
> >>> response will not be sent back until the metadata has been propagated
> to
> >>> the whole cluster.
> >>
> >> Hmm.  It seems like this would require additional RPCs or changes to
> >> existing RPCs on the server.  So we should handle this in a follow-on
> >> KIP, I think.
> >>
> >>
> > I agree for STRICT_BLOCKING, while for BLOCKING, it is already supported
> as
> > of today I think, and Streams' KafkaClient is using that mechanism as
> well.
> >
> >
> >>>
> >>> 3. I'm wondering what's the usage of "public Map >>> Try> getAllGroups()", or rather, would it be more
> >>> useful to get a specific group information given the group id?
> Otherwise
> >>> we
> >>> need to query each one of the coordinator.
> >>
> >> That's a good point.  We should have an API that gets information about
> >> a specific group, by querying only the coordinator for that group.  By
> >> the way, what specific group information should we expose, besides name
> >> and protocolType?
> >>
> >>
> > I think these can all be returned?
> >
> > (groupID, protocolType, generationID, state, members: [memberID,
> > clientHost], leaderID (nullable) )
> >
> >
> >>>
> >>> 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
> >>> with a KafkaAdminClient impl, this at least allows easier mocks for
> unit
> >>> testing.
> >>
> >> Yeah, I agree.  Hopefully that will also make it a little clearer what
> >> the boundary is between the internal functions and classes and the
> >> public API.  I'll update the KIP accordingly.
> >>
> >> thanks,
> >> Colin
> >>
> >>>
> >>> Guozhang
> >>>
> >>>
> >>>
> >>> On Fri, Feb 3, 2017 at 10:40 AM, Colin McCabe 
> >> wrote:
> >>>
>  On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> > Hi Colin,
> >
> > Thanks for the KIP, great to make progress on this. I have some
> >> initial
> > comments, will post more later:
> >
> > 1. We have KafkaProducer that implements the Producer interface and
> > KafkaConsumer that implements the Consumer interface. Maybe we could
> >> have
> > KafkaAdminClient that implements the AdminClient interface? Or maybe
> >> just
> > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't
> >> think
> > we
> > should worry about a name clash with the internal AdminClient
> >> written in
> > Scala. That will go away soon enough and choosing a good name for the
> > public class is what matters.
> 
>  Hi Ismael,
> 
>  Thanks for taking a look.
> 
>  I guess my thought process was that users might find it confusing if
> >> the
>  public API and the old private API had the same name.  "What do you
>  mean, I have to upgrade to release X to get AdminClient, I have it
> >> right
>  here?"  I do have a slight preference for the shorter name, though, so
>  if this isn't a worry, we can change it to AdminClient.
> 
> >
> > 2. We should include the proposed package name in the KIP
> > (probably org.apache.kafka.clients.admin?).
> 
>  Good idea.  I will add the package name to 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Jianbin Wei
In the specify group information, can we also return information like partition 
assignment for each member, the lag/offset of each member/partition?  It would 
be useful for Ops/Admin regarding the health of the consumer group.

Regards,

-- Jianbin

> On Feb 6, 2017, at 13:54, Guozhang Wang  wrote:
> 
> Some follow-up on 2) / 3) below.
> 
> On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe  > wrote:
> 
>> On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
>>> Thanks for the proposal Colin. A few comments below:
>> 
>> Thanks for taking a look, Guozhang.
>> 
>>> 
>>> 1. There are a couple of classes that looks new to me but not defined
>>> anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is
>>> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
>>> GroupOverview.
>>> Could you provide their class definitions?
>> 
>> Good point.  I will add them in the KIP.
>> 
>> NodeApiVersions is at
>> ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
>> 
>>> 
>>> 2. In Streams, we would like to replace its own `
>>> org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
>>> with
>>> this new admin client.  One additional request though, is that for create
>>> /
>>> delete topics, we'd like to use a different "flag" as BLOCKING, which
>>> means
>>> the response will not be sent back until the controller has updated its
>>> own
>>> metadata cache for the topic, and even STRICT_BLOCKING, which means the
>>> response will not be sent back until the metadata has been propagated to
>>> the whole cluster.
>> 
>> Hmm.  It seems like this would require additional RPCs or changes to
>> existing RPCs on the server.  So we should handle this in a follow-on
>> KIP, I think.
>> 
>> 
> I agree for STRICT_BLOCKING, while for BLOCKING, it is already supported as
> of today I think, and Streams' KafkaClient is using that mechanism as well.
> 
> 
>>> 
>>> 3. I'm wondering what's the usage of "public Map>> Try> getAllGroups()", or rather, would it be more
>>> useful to get a specific group information given the group id? Otherwise
>>> we
>>> need to query each one of the coordinator.
>> 
>> That's a good point.  We should have an API that gets information about
>> a specific group, by querying only the coordinator for that group.  By
>> the way, what specific group information should we expose, besides name
>> and protocolType?
>> 
>> 
> I think these can all be returned?
> 
> (groupID, protocolType, generationID, state, members: [memberID,
> clientHost], leaderID (nullable) )
> 
> 
>>> 
>>> 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
>>> with a KafkaAdminClient impl, this at least allows easier mocks for unit
>>> testing.
>> 
>> Yeah, I agree.  Hopefully that will also make it a little clearer what
>> the boundary is between the internal functions and classes and the
>> public API.  I'll update the KIP accordingly.
>> 
>> thanks,
>> Colin
>> 
>>> 
>>> Guozhang
>>> 
>>> 
>>> 
>>> On Fri, Feb 3, 2017 at 10:40 AM, Colin McCabe 
>> wrote:
>>> 
 On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> Hi Colin,
> 
> Thanks for the KIP, great to make progress on this. I have some
>> initial
> comments, will post more later:
> 
> 1. We have KafkaProducer that implements the Producer interface and
> KafkaConsumer that implements the Consumer interface. Maybe we could
>> have
> KafkaAdminClient that implements the AdminClient interface? Or maybe
>> just
> KafkaAdmin. Not sure, some ideas for consideration. Also, I don't
>> think
> we
> should worry about a name clash with the internal AdminClient
>> written in
> Scala. That will go away soon enough and choosing a good name for the
> public class is what matters.
 
 Hi Ismael,
 
 Thanks for taking a look.
 
 I guess my thought process was that users might find it confusing if
>> the
 public API and the old private API had the same name.  "What do you
 mean, I have to upgrade to release X to get AdminClient, I have it
>> right
 here?"  I do have a slight preference for the shorter name, though, so
 if this isn't a worry, we can change it to AdminClient.
 
> 
> 2. We should include the proposed package name in the KIP
> (probably org.apache.kafka.clients.admin?).
 
 Good idea.  I will add the package name to the KIP.
 
> 
> 3. It would be good to list the supported configs.
 
 OK
 
> 
> 4. KIP-107, which passed the vote, specifies the introduction of a
>> method
> to AdminClient with the following signature. We should figure out
>> how it
> would look given this proposal.
> 
> Future>
> purgeDataBefore(Map offsetForPartition)
> 
> 5. I am not sure about rejecting the 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Guozhang Wang
Some follow-up on 2) / 3) below.

On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe  wrote:

> On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> > Thanks for the proposal Colin. A few comments below:
>
> Thanks for taking a look, Guozhang.
>
> >
> > 1. There are a couple of classes that looks new to me but not defined
> > anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is
> > this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> > GroupOverview.
> > Could you provide their class definitions?
>
> Good point.  I will add them in the KIP.
>
> NodeApiVersions is at
> ./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java
>
> >
> > 2. In Streams, we would like to replace its own `
> > org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> > with
> > this new admin client.  One additional request though, is that for create
> > /
> > delete topics, we'd like to use a different "flag" as BLOCKING, which
> > means
> > the response will not be sent back until the controller has updated its
> > own
> > metadata cache for the topic, and even STRICT_BLOCKING, which means the
> > response will not be sent back until the metadata has been propagated to
> > the whole cluster.
>
> Hmm.  It seems like this would require additional RPCs or changes to
> existing RPCs on the server.  So we should handle this in a follow-on
> KIP, I think.
>
>
I agree for STRICT_BLOCKING, while for BLOCKING, it is already supported as
of today I think, and Streams' KafkaClient is using that mechanism as well.


> >
> > 3. I'm wondering what's the usage of "public Map > Try> getAllGroups()", or rather, would it be more
> > useful to get a specific group information given the group id? Otherwise
> > we
> > need to query each one of the coordinator.
>
> That's a good point.  We should have an API that gets information about
> a specific group, by querying only the coordinator for that group.  By
> the way, what specific group information should we expose, besides name
> and protocolType?
>
>
I think these can all be returned?

(groupID, protocolType, generationID, state, members: [memberID,
clientHost], leaderID (nullable) )


> >
> > 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
> > with a KafkaAdminClient impl, this at least allows easier mocks for unit
> > testing.
>
> Yeah, I agree.  Hopefully that will also make it a little clearer what
> the boundary is between the internal functions and classes and the
> public API.  I'll update the KIP accordingly.
>
> thanks,
> Colin
>
> >
> > Guozhang
> >
> >
> >
> > On Fri, Feb 3, 2017 at 10:40 AM, Colin McCabe 
> wrote:
> >
> > > On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> > > > Hi Colin,
> > > >
> > > > Thanks for the KIP, great to make progress on this. I have some
> initial
> > > > comments, will post more later:
> > > >
> > > > 1. We have KafkaProducer that implements the Producer interface and
> > > > KafkaConsumer that implements the Consumer interface. Maybe we could
> have
> > > > KafkaAdminClient that implements the AdminClient interface? Or maybe
> just
> > > > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't
> think
> > > > we
> > > > should worry about a name clash with the internal AdminClient
> written in
> > > > Scala. That will go away soon enough and choosing a good name for the
> > > > public class is what matters.
> > >
> > > Hi Ismael,
> > >
> > > Thanks for taking a look.
> > >
> > > I guess my thought process was that users might find it confusing if
> the
> > > public API and the old private API had the same name.  "What do you
> > > mean, I have to upgrade to release X to get AdminClient, I have it
> right
> > > here?"  I do have a slight preference for the shorter name, though, so
> > > if this isn't a worry, we can change it to AdminClient.
> > >
> > > >
> > > > 2. We should include the proposed package name in the KIP
> > > > (probably org.apache.kafka.clients.admin?).
> > >
> > > Good idea.  I will add the package name to the KIP.
> > >
> > > >
> > > > 3. It would be good to list the supported configs.
> > >
> > > OK
> > >
> > > >
> > > > 4. KIP-107, which passed the vote, specifies the introduction of a
> method
> > > > to AdminClient with the following signature. We should figure out
> how it
> > > > would look given this proposal.
> > > >
> > > > Future>
> > > > purgeDataBefore(Map offsetForPartition)
> > > >
> > > > 5. I am not sure about rejecting the Futures-based API. I think I
> would
> > > > prefer that, personally. Grant had an interesting idea of not
> exposing
> > > > the
> > > > batch methods in the AdminClient to start with to keep it simple and
> > > > relying on a Future based API to make it easy for users to run things
> > > > concurrently. This is consistent with the producer...
> > >
> > > So, there are two ways that an 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Jeff Widman
For individual consumer groups, it would be nice if the admin client made
it possible to fetch consumer offsets for the entire consumer group. Then
we don't have to manually assemble this outside of the admin client
interface.

On Feb 6, 2017 11:41 AM, "Colin McCabe"  wrote:

> On Fri, Feb 3, 2017, at 16:57, Dong Lin wrote:
> > Thanks for the reply, Colin. I have some comments inline.
>
> Hi Dong L.,
>
> >
> > In addition, I also have some comments regarding the Future() in response
> > to your latest email. As Ismael mentioned, we have added
> > purgeDataBefore()
> > API in AdminClient. This API returns Future() that allows user to purge
> > data in either syn or async manner. And we have presented use-case for
> > both
> > syn and async usage of this API in the discussion thread of KIP-107. I
> > think we should at least return a Future() object in this case, right?
>
> Hmm.  I didn't see any discussion in the KIP-107 [DISCUSS] thread of why
> a Future<> based API was proposed.  It seemed like the discussion
> focused on other issues (or maybe I missed the an email thread?)
>
> >
> > As you mentioned, we can transform a blocking API into a Futures-based
> > API
> > by using a thread pool. But thread pool seems inconvenient as compared to
> > using future().get() which transform a Future-based API into a blocking
> > API. Would it be a good reason to return Future() for all those API where
> > we need both syn and async mode?
>
> That's a good point.  It's easier to build a sync API on top of Futures
> than the reverse.
>
> >
> >
> > On Fri, Feb 3, 2017 at 10:20 AM, Colin McCabe 
> wrote:
> >
> > > On Thu, Feb 2, 2017, at 17:54, Dong Lin wrote:
> > > > Hey Colin,
> > > >
> > > > Thanks for the KIP. I have a few comments below:
> > > >
> > > > - I share similar view with Ismael that a Future-based API is better.
> > > > PurgeDataFrom() is an example API that user may want to do it
> > > > asynchronously even though there is only one request in flight at a
> time.
> > > > In the future we may also have some admin operation that allows user
> to
> > > > move replica from one broker to another, which also needs to work in
> both
> > > > sync and async style. It seems more generic to return Future for
> any
> > > > API
> > > > that requires both mode.
> > > >
> > > > - I am not sure if it is the cleanest way to have enum classes
> > > > CreateTopicsFlags and DeleteTopicsFlags. Are we going to create such
> > > > class
> > > > for every future API that requires configuration? It may be more
> generic
> > > > to
> > > > provide Map to any admin API that operates on multiple
> > > > entries.
> > > > For example, deleteTopic(Map). And it can be
> Map > > > Properties> for those API that requires multiple configs per entry.
> And
> > > > we
> > > > can provide default value, doc, config name for those API as we do
> > > > with AbstractConfig.
> > >
> > > Thanks for the comments, Dong L.
> > >
> > > EnumSet, EnumSet, and so forth
> are
> > > type-safe ways for the user to pass a set of boolean flags to the
> > > function.  It is basically a replacement for having an api like
> > > createTopics(, boolean nonblocking, boolean validateOnly).  It is
> > > preferrable to having the boolean flags because it's immediately clear
> > > when reading the code what createTopics(...,
> > > CreateTopicsFlags.VALIDATE_ONLY) means, whereas it is not immediately
> > > clear what createTopics(..., false, true) means.  It also prevents
> > > having lots of function overloads over time, which becomes confusing
> for
> > > users.  The EnumSet is not intended as a replacement for all possible
> > > future arguments we might add, but just an easy way to add more boolean
> > > arguments later without adding messy function overloads or type
> > > signatures with many booleans.
> > >
> > > Map is not type-safe, and I don't think we should use it in
> > > our APIs.  Instead, we should just add function overloads if necessary.
> > >
> >
> > I agree that using EnumSet is type safe.
> >
> >
> > >
> > > >
> > > > - I not sure if "Try" is very intuitive to Java developer. Is there
> any
> > > > counterpart of scala's "Try" in java
> > >
> > > Unfortunately, there is no equivalent to Try in the standard Java
> > > library.  That is the first place I checked, and I spent a while
> > > searching.  The closest is probably Java 8's Optional.  However,
> > > Optional just allows us to express Some(thing) or None, so callers
> would
> > > not be able to determine what the error was.
> >
> >
> > > > We actually have quite a few
> > > > existing
> > > > classes in Kafka that address the same problem, such as
> > > > ProduceRequestResult, LogAppendResult etc. Maybe we can follow the
> same
> > > > conversion and use *Result as this class name.
> > >
> > > Hmm.  ProduceRequestResult and LogAppendResult just store an exception
> > > alongside the data, and make the 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Colin McCabe
On Fri, Feb 3, 2017, at 16:57, Dong Lin wrote:
> Thanks for the reply, Colin. I have some comments inline.

Hi Dong L.,

> 
> In addition, I also have some comments regarding the Future() in response
> to your latest email. As Ismael mentioned, we have added
> purgeDataBefore()
> API in AdminClient. This API returns Future() that allows user to purge
> data in either syn or async manner. And we have presented use-case for
> both
> syn and async usage of this API in the discussion thread of KIP-107. I
> think we should at least return a Future() object in this case, right?

Hmm.  I didn't see any discussion in the KIP-107 [DISCUSS] thread of why
a Future<> based API was proposed.  It seemed like the discussion
focused on other issues (or maybe I missed the an email thread?)

> 
> As you mentioned, we can transform a blocking API into a Futures-based
> API
> by using a thread pool. But thread pool seems inconvenient as compared to
> using future().get() which transform a Future-based API into a blocking
> API. Would it be a good reason to return Future() for all those API where
> we need both syn and async mode?

That's a good point.  It's easier to build a sync API on top of Futures
than the reverse.

> 
> 
> On Fri, Feb 3, 2017 at 10:20 AM, Colin McCabe  wrote:
> 
> > On Thu, Feb 2, 2017, at 17:54, Dong Lin wrote:
> > > Hey Colin,
> > >
> > > Thanks for the KIP. I have a few comments below:
> > >
> > > - I share similar view with Ismael that a Future-based API is better.
> > > PurgeDataFrom() is an example API that user may want to do it
> > > asynchronously even though there is only one request in flight at a time.
> > > In the future we may also have some admin operation that allows user to
> > > move replica from one broker to another, which also needs to work in both
> > > sync and async style. It seems more generic to return Future for any
> > > API
> > > that requires both mode.
> > >
> > > - I am not sure if it is the cleanest way to have enum classes
> > > CreateTopicsFlags and DeleteTopicsFlags. Are we going to create such
> > > class
> > > for every future API that requires configuration? It may be more generic
> > > to
> > > provide Map to any admin API that operates on multiple
> > > entries.
> > > For example, deleteTopic(Map). And it can be Map > > Properties> for those API that requires multiple configs per entry. And
> > > we
> > > can provide default value, doc, config name for those API as we do
> > > with AbstractConfig.
> >
> > Thanks for the comments, Dong L.
> >
> > EnumSet, EnumSet, and so forth are
> > type-safe ways for the user to pass a set of boolean flags to the
> > function.  It is basically a replacement for having an api like
> > createTopics(, boolean nonblocking, boolean validateOnly).  It is
> > preferrable to having the boolean flags because it's immediately clear
> > when reading the code what createTopics(...,
> > CreateTopicsFlags.VALIDATE_ONLY) means, whereas it is not immediately
> > clear what createTopics(..., false, true) means.  It also prevents
> > having lots of function overloads over time, which becomes confusing for
> > users.  The EnumSet is not intended as a replacement for all possible
> > future arguments we might add, but just an easy way to add more boolean
> > arguments later without adding messy function overloads or type
> > signatures with many booleans.
> >
> > Map is not type-safe, and I don't think we should use it in
> > our APIs.  Instead, we should just add function overloads if necessary.
> >
> 
> I agree that using EnumSet is type safe.
> 
> 
> >
> > >
> > > - I not sure if "Try" is very intuitive to Java developer. Is there any
> > > counterpart of scala's "Try" in java
> >
> > Unfortunately, there is no equivalent to Try in the standard Java
> > library.  That is the first place I checked, and I spent a while
> > searching.  The closest is probably Java 8's Optional.  However,
> > Optional just allows us to express Some(thing) or None, so callers would
> > not be able to determine what the error was.
> 
> 
> > > We actually have quite a few
> > > existing
> > > classes in Kafka that address the same problem, such as
> > > ProduceRequestResult, LogAppendResult etc. Maybe we can follow the same
> > > conversion and use *Result as this class name.
> >
> > Hmm.  ProduceRequestResult and LogAppendResult just store an exception
> > alongside the data, and make the caller responsible for checking whether
> > the exception is null (or None) before looking at the data.  I don't
> > think this is a good solution, because if the user forgets to do this,
> > they will interpret whatever is in the data fields as valid, even when
> > it is not.  ProduceRequestResult and LogAppendResult are also internal
> > classes, not user-facing APIs, so we did not spend time thinking about
> > how to make them easy to use for end-users.
> >
> 
> I am not actually suggesting to use 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Colin McCabe
On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> Thanks for the proposal Colin. A few comments below:
> 
> 1. There are a couple of classes that looks new to me but not defined
> anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is
> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> GroupOverview.
> Could you provide their class definitions?
> 
> 2. In Streams, we would like to replace its own `
> org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> with
> this new admin client.  One additional request though, is that for create
> /
> delete topics, we'd like to use a different "flag" as BLOCKING, which
> means
> the response will not be sent back until the controller has updated its
> own
> metadata cache for the topic, and even STRICT_BLOCKING, which means the
> response will not be sent back until the metadata has been propagated to
> the whole cluster.
> 
> 3. I'm wondering what's the usage of "public Map Try> getAllGroups()"

Oh, by the way, this method is a copy of the AdminClient.scala
"listAllConsumerGroups" method, which is used in ConsumerGroupCommand to
provide that output.  So that's where the usage is.  But I agree that it
would be nice to be able to get information about just one group as
well.

cheers,
Colin

> or rather, would it be more
> useful to get a specific group information given the group id? Otherwise
> we
> need to query each one of the coordinator.
> 
> 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
> with a KafkaAdminClient impl, this at least allows easier mocks for unit
> testing.
> 
> Guozhang
> 
> 
> 
> On Fri, Feb 3, 2017 at 10:40 AM, Colin McCabe  wrote:
> 
> > On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> > > Hi Colin,
> > >
> > > Thanks for the KIP, great to make progress on this. I have some initial
> > > comments, will post more later:
> > >
> > > 1. We have KafkaProducer that implements the Producer interface and
> > > KafkaConsumer that implements the Consumer interface. Maybe we could have
> > > KafkaAdminClient that implements the AdminClient interface? Or maybe just
> > > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think
> > > we
> > > should worry about a name clash with the internal AdminClient written in
> > > Scala. That will go away soon enough and choosing a good name for the
> > > public class is what matters.
> >
> > Hi Ismael,
> >
> > Thanks for taking a look.
> >
> > I guess my thought process was that users might find it confusing if the
> > public API and the old private API had the same name.  "What do you
> > mean, I have to upgrade to release X to get AdminClient, I have it right
> > here?"  I do have a slight preference for the shorter name, though, so
> > if this isn't a worry, we can change it to AdminClient.
> >
> > >
> > > 2. We should include the proposed package name in the KIP
> > > (probably org.apache.kafka.clients.admin?).
> >
> > Good idea.  I will add the package name to the KIP.
> >
> > >
> > > 3. It would be good to list the supported configs.
> >
> > OK
> >
> > >
> > > 4. KIP-107, which passed the vote, specifies the introduction of a method
> > > to AdminClient with the following signature. We should figure out how it
> > > would look given this proposal.
> > >
> > > Future>
> > > purgeDataBefore(Map offsetForPartition)
> > >
> > > 5. I am not sure about rejecting the Futures-based API. I think I would
> > > prefer that, personally. Grant had an interesting idea of not exposing
> > > the
> > > batch methods in the AdminClient to start with to keep it simple and
> > > relying on a Future based API to make it easy for users to run things
> > > concurrently. This is consistent with the producer...
> >
> > So, there are two ways that an operation can be "async" here which are
> > very separate.
> >
> > There is "async on the server."  This basically means that we tell the
> > server to do something and don't wait for a confirmation that it
> > succeeded.  For example, in the current proposal, users can call
> > createTopic(new Topic(...), CreateTopicFlags.NONBLOCKING).  The call
> > will wait for the server to get the request, which will go into
> > purgatory.  Later on, the request may succeed or fail, but the client
> > won't know either way.  In RPC terms, this means we set the timeout
> > value to 0.
> >
> > Then there is "async on the client."  This just means that the client
> > thread doesn't block-- instead, it gets back a Future (or similar
> > object).  What this boils down to in terms of implementation is that a
> > message gets put on some queue somewhere and the client thread continues
> > running.
> >
> > "async on the client" tends to be good when you want to churn out a ton
> > of requests without using lots of threads.  However, it is more
> > confusing mental model for most programmers.
> >
> > You can easily 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Colin McCabe
On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote:
> Thanks for the proposal Colin. A few comments below:

Thanks for taking a look, Guozhang.

> 
> 1. There are a couple of classes that looks new to me but not defined
> anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is
> this a wrapper of MetadataResponse.TopicMetadata?), NodeApiVersions,
> GroupOverview.
> Could you provide their class definitions?

Good point.  I will add them in the KIP.

NodeApiVersions is at
./clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java

> 
> 2. In Streams, we would like to replace its own `
> org.apache.kafka.streams.processor.internals.StreamsKafkaClient` class
> with
> this new admin client.  One additional request though, is that for create
> /
> delete topics, we'd like to use a different "flag" as BLOCKING, which
> means
> the response will not be sent back until the controller has updated its
> own
> metadata cache for the topic, and even STRICT_BLOCKING, which means the
> response will not be sent back until the metadata has been propagated to
> the whole cluster.

Hmm.  It seems like this would require additional RPCs or changes to
existing RPCs on the server.  So we should handle this in a follow-on
KIP, I think.

> 
> 3. I'm wondering what's the usage of "public Map Try> getAllGroups()", or rather, would it be more
> useful to get a specific group information given the group id? Otherwise
> we
> need to query each one of the coordinator.

That's a good point.  We should have an API that gets information about
a specific group, by querying only the coordinator for that group.  By
the way, what specific group information should we expose, besides name
and protocolType?

> 
> 4. I'm +1 with Ismael's suggestion for having the AdminClient interface
> with a KafkaAdminClient impl, this at least allows easier mocks for unit
> testing.

Yeah, I agree.  Hopefully that will also make it a little clearer what
the boundary is between the internal functions and classes and the
public API.  I'll update the KIP accordingly.

thanks,
Colin

> 
> Guozhang
> 
> 
> 
> On Fri, Feb 3, 2017 at 10:40 AM, Colin McCabe  wrote:
> 
> > On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> > > Hi Colin,
> > >
> > > Thanks for the KIP, great to make progress on this. I have some initial
> > > comments, will post more later:
> > >
> > > 1. We have KafkaProducer that implements the Producer interface and
> > > KafkaConsumer that implements the Consumer interface. Maybe we could have
> > > KafkaAdminClient that implements the AdminClient interface? Or maybe just
> > > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think
> > > we
> > > should worry about a name clash with the internal AdminClient written in
> > > Scala. That will go away soon enough and choosing a good name for the
> > > public class is what matters.
> >
> > Hi Ismael,
> >
> > Thanks for taking a look.
> >
> > I guess my thought process was that users might find it confusing if the
> > public API and the old private API had the same name.  "What do you
> > mean, I have to upgrade to release X to get AdminClient, I have it right
> > here?"  I do have a slight preference for the shorter name, though, so
> > if this isn't a worry, we can change it to AdminClient.
> >
> > >
> > > 2. We should include the proposed package name in the KIP
> > > (probably org.apache.kafka.clients.admin?).
> >
> > Good idea.  I will add the package name to the KIP.
> >
> > >
> > > 3. It would be good to list the supported configs.
> >
> > OK
> >
> > >
> > > 4. KIP-107, which passed the vote, specifies the introduction of a method
> > > to AdminClient with the following signature. We should figure out how it
> > > would look given this proposal.
> > >
> > > Future>
> > > purgeDataBefore(Map offsetForPartition)
> > >
> > > 5. I am not sure about rejecting the Futures-based API. I think I would
> > > prefer that, personally. Grant had an interesting idea of not exposing
> > > the
> > > batch methods in the AdminClient to start with to keep it simple and
> > > relying on a Future based API to make it easy for users to run things
> > > concurrently. This is consistent with the producer...
> >
> > So, there are two ways that an operation can be "async" here which are
> > very separate.
> >
> > There is "async on the server."  This basically means that we tell the
> > server to do something and don't wait for a confirmation that it
> > succeeded.  For example, in the current proposal, users can call
> > createTopic(new Topic(...), CreateTopicFlags.NONBLOCKING).  The call
> > will wait for the server to get the request, which will go into
> > purgatory.  Later on, the request may succeed or fail, but the client
> > won't know either way.  In RPC terms, this means we set the timeout
> > value to 0.
> >
> > Then there is "async on the client."  This just means that 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-05 Thread Manikumar
+1 for ismael's suggestion.  grouping of methods by the relevant resource
will simply the
API. In future, we will be adding delegation token related operations to
admin client.
I can imagine methods like adminClient.token().create(...),
adminClient.token().renew(...), etc..


Thanks,
Manikumar

On Sun, Feb 5, 2017 at 7:27 PM, Ismael Juma  wrote:

> Hi Colin,
>
> I was thinking about the API and the fact that the AdminClient will be used
> by a variety of somewhat unrelated things (unlike the Consumer and
> Producer), so I think we should consider an approach where not all the
> methods are at the top-level. One idea is that you could perform operations
> on topics by doing something like `adminClient.topics().create(...)`,
> `adminClient.topics().delete(...)`, `adminClient.topics().describe(...)`,
> etc. This is just an initial sketch to describe the idea, but I think it
> would be a nice way to group methods by the relevant resource. It would
> also make it easier to enforce consistency by using shared interfaces for
> the types returned by the resource method (e.g. `topics()`, `acls()`,
> etc.).
>
> One additional comment inline.
>
> On Fri, Feb 3, 2017 at 6:40 PM, Colin McCabe  wrote:
> >
> > I guess my thought process was that users might find it confusing if the
> > public API and the old private API had the same name.  "What do you
> > mean, I have to upgrade to release X to get AdminClient, I have it right
> > here?"  I do have a slight preference for the shorter name, though, so
> > if this isn't a worry, we can change it to AdminClient.
> >
>
> Yes, I don't think this is a worry. The package name and implementation
> language are different, so it's easy enough to distinguish. We should not
> choose a worse name for a public class because of an internal class, as a
> general rule, in my opinion.
>
> More to follow later.
>
> Ismael
>


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-05 Thread Ismael Juma
Hi Colin,

I was thinking about the API and the fact that the AdminClient will be used
by a variety of somewhat unrelated things (unlike the Consumer and
Producer), so I think we should consider an approach where not all the
methods are at the top-level. One idea is that you could perform operations
on topics by doing something like `adminClient.topics().create(...)`,
`adminClient.topics().delete(...)`, `adminClient.topics().describe(...)`,
etc. This is just an initial sketch to describe the idea, but I think it
would be a nice way to group methods by the relevant resource. It would
also make it easier to enforce consistency by using shared interfaces for
the types returned by the resource method (e.g. `topics()`, `acls()`, etc.).

One additional comment inline.

On Fri, Feb 3, 2017 at 6:40 PM, Colin McCabe  wrote:
>
> I guess my thought process was that users might find it confusing if the
> public API and the old private API had the same name.  "What do you
> mean, I have to upgrade to release X to get AdminClient, I have it right
> here?"  I do have a slight preference for the shorter name, though, so
> if this isn't a worry, we can change it to AdminClient.
>

Yes, I don't think this is a worry. The package name and implementation
language are different, so it's easy enough to distinguish. We should not
choose a worse name for a public class because of an internal class, as a
general rule, in my opinion.

More to follow later.

Ismael


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-03 Thread Dong Lin
Thanks for the reply, Colin. I have some comments inline.

In addition, I also have some comments regarding the Future() in response
to your latest email. As Ismael mentioned, we have added purgeDataBefore()
API in AdminClient. This API returns Future() that allows user to purge
data in either syn or async manner. And we have presented use-case for both
syn and async usage of this API in the discussion thread of KIP-107. I
think we should at least return a Future() object in this case, right?

As you mentioned, we can transform a blocking API into a Futures-based API
by using a thread pool. But thread pool seems inconvenient as compared to
using future().get() which transform a Future-based API into a blocking
API. Would it be a good reason to return Future() for all those API where
we need both syn and async mode?


On Fri, Feb 3, 2017 at 10:20 AM, Colin McCabe  wrote:

> On Thu, Feb 2, 2017, at 17:54, Dong Lin wrote:
> > Hey Colin,
> >
> > Thanks for the KIP. I have a few comments below:
> >
> > - I share similar view with Ismael that a Future-based API is better.
> > PurgeDataFrom() is an example API that user may want to do it
> > asynchronously even though there is only one request in flight at a time.
> > In the future we may also have some admin operation that allows user to
> > move replica from one broker to another, which also needs to work in both
> > sync and async style. It seems more generic to return Future for any
> > API
> > that requires both mode.
> >
> > - I am not sure if it is the cleanest way to have enum classes
> > CreateTopicsFlags and DeleteTopicsFlags. Are we going to create such
> > class
> > for every future API that requires configuration? It may be more generic
> > to
> > provide Map to any admin API that operates on multiple
> > entries.
> > For example, deleteTopic(Map). And it can be Map > Properties> for those API that requires multiple configs per entry. And
> > we
> > can provide default value, doc, config name for those API as we do
> > with AbstractConfig.
>
> Thanks for the comments, Dong L.
>
> EnumSet, EnumSet, and so forth are
> type-safe ways for the user to pass a set of boolean flags to the
> function.  It is basically a replacement for having an api like
> createTopics(, boolean nonblocking, boolean validateOnly).  It is
> preferrable to having the boolean flags because it's immediately clear
> when reading the code what createTopics(...,
> CreateTopicsFlags.VALIDATE_ONLY) means, whereas it is not immediately
> clear what createTopics(..., false, true) means.  It also prevents
> having lots of function overloads over time, which becomes confusing for
> users.  The EnumSet is not intended as a replacement for all possible
> future arguments we might add, but just an easy way to add more boolean
> arguments later without adding messy function overloads or type
> signatures with many booleans.
>
> Map is not type-safe, and I don't think we should use it in
> our APIs.  Instead, we should just add function overloads if necessary.
>

I agree that using EnumSet is type safe.


>
> >
> > - I not sure if "Try" is very intuitive to Java developer. Is there any
> > counterpart of scala's "Try" in java
>
> Unfortunately, there is no equivalent to Try in the standard Java
> library.  That is the first place I checked, and I spent a while
> searching.  The closest is probably Java 8's Optional.  However,
> Optional just allows us to express Some(thing) or None, so callers would
> not be able to determine what the error was.


> > We actually have quite a few
> > existing
> > classes in Kafka that address the same problem, such as
> > ProduceRequestResult, LogAppendResult etc. Maybe we can follow the same
> > conversion and use *Result as this class name.
>
> Hmm.  ProduceRequestResult and LogAppendResult just store an exception
> alongside the data, and make the caller responsible for checking whether
> the exception is null (or None) before looking at the data.  I don't
> think this is a good solution, because if the user forgets to do this,
> they will interpret whatever is in the data fields as valid, even when
> it is not.  ProduceRequestResult and LogAppendResult are also internal
> classes, not user-facing APIs, so we did not spend time thinking about
> how to make them easy to use for end-users.
>

I am not actually suggesting to use LogAppendResult directly. I am
wondering if it would be more intuitive for developer to name this class
something like OperationResult instead of Try. The OperationResult can
store whatever we want to store with current "Try" class you proposed in
the KIP. It is just my personal opinion that "Try" doesn't directly tell me
what the class actually does. But I am not too worried about it if you
think "Try" is better.


>
> >
> > - How are we going to allow user to specify timeout for blocking APIs
> > such
> > as deleteTopic? Is this configured per AdminClient, or 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-03 Thread Colin McCabe
On Thu, Feb 2, 2017, at 15:02, Ismael Juma wrote:
> Hi Colin,
> 
> Thanks for the KIP, great to make progress on this. I have some initial
> comments, will post more later:
> 
> 1. We have KafkaProducer that implements the Producer interface and
> KafkaConsumer that implements the Consumer interface. Maybe we could have
> KafkaAdminClient that implements the AdminClient interface? Or maybe just
> KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think
> we
> should worry about a name clash with the internal AdminClient written in
> Scala. That will go away soon enough and choosing a good name for the
> public class is what matters.

Hi Ismael,

Thanks for taking a look.

I guess my thought process was that users might find it confusing if the
public API and the old private API had the same name.  "What do you
mean, I have to upgrade to release X to get AdminClient, I have it right
here?"  I do have a slight preference for the shorter name, though, so
if this isn't a worry, we can change it to AdminClient.

> 
> 2. We should include the proposed package name in the KIP
> (probably org.apache.kafka.clients.admin?).

Good idea.  I will add the package name to the KIP.

> 
> 3. It would be good to list the supported configs.

OK

> 
> 4. KIP-107, which passed the vote, specifies the introduction of a method
> to AdminClient with the following signature. We should figure out how it
> would look given this proposal.
> 
> Future>
> purgeDataBefore(Map offsetForPartition)
> 
> 5. I am not sure about rejecting the Futures-based API. I think I would
> prefer that, personally. Grant had an interesting idea of not exposing
> the
> batch methods in the AdminClient to start with to keep it simple and
> relying on a Future based API to make it easy for users to run things
> concurrently. This is consistent with the producer... 

So, there are two ways that an operation can be "async" here which are
very separate.

There is "async on the server."  This basically means that we tell the
server to do something and don't wait for a confirmation that it
succeeded.  For example, in the current proposal, users can call
createTopic(new Topic(...), CreateTopicFlags.NONBLOCKING).  The call
will wait for the server to get the request, which will go into
purgatory.  Later on, the request may succeed or fail, but the client
won't know either way.  In RPC terms, this means we set the timeout
value to 0.

Then there is "async on the client."  This just means that the client
thread doesn't block-- instead, it gets back a Future (or similar
object).  What this boils down to in terms of implementation is that a
message gets put on some queue somewhere and the client thread continues
running.

"async on the client" tends to be good when you want to churn out a ton
of requests without using lots of threads.  However, it is more
confusing mental model for most programmers.

You can easily translate a Futures-based API into a blocking API by
having blocking shims that just call create the Future and call get(). 
Similarly, you can transform a blocking API into a Futures-based API by
using a thread pool.  Thread pools use resources, though, whereas having
function shims does not.

I haven't seen any discussion here about what we gain here by using a
Futures-based API.  It makes sense to use Futures in the Producer, since
they're more flexible, and users are potentially creating lots and lots
of messages.  I'm not sure if users would want to do lots and lots of
admin operations with a single thread.  I'd be curious to hear a little
more from potential end-users about what API would be most flexible and
usable for them.  I'm open to ideas.

best,
Colin

> 
> Ismael
> 
> On Thu, Feb 2, 2017 at 6:54 PM, Colin McCabe  wrote:
> 
> > Hi all,
> >
> > I wrote up a Kafka improvement proposal for adding an
> > AdministrativeClient interface.  This is a continuation of the work on
> > KIP-4 towards centralized administrative operations.  Please check out
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public+
> > AdministrativeClient+API+for+Kafka+admin+operations
> >
> > regards,
> > Colin
> >


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-03 Thread Colin McCabe
On Thu, Feb 2, 2017, at 21:45, Becket Qin wrote:
> Hi Colin,
> 
> Thanks for the KIP. An admin client is probably a must after we block
> direct access to ZK. Some comments and thoughts below:
> 
> 1. Do we have a clear scope for the admin client? It might be worth
> thinking about the entire user experience of the admin client. Ideally we
> may want to have a single client to do all the administrative work
> instead
> of having multiple ways to do different things. For example, do we want
> to
> add topic configurations change API in the admin client? What about
> partition movements and preferred leader election? Those are also
> administrative tasks which seem reasonable to be integrated into the
> admin
> client.

Thanks for the comments, Becket!

I agree that topic configuration change should be in the administrative
client.  I have not thought about partition movement or preferred leader
election.  It probably makes sense to put them in the client as well,
but we should probably have a longer discussion about those features
later when someone is ready to implement them ;)

best,
Colin

> 
> 2. Regarding the Future based async Ops v.s. batching of Ops, I would
> prefer supporting batching if possible. That usually introduce much less
> overhead when doing some big operations, e.g. in controller we have been
> putting quite some efforts to batch the requests. For admin client, my
> understanding is that the operations are:
> a. rare and potentially big
> b. likely OK to block (it would be good to see some use cases where a
> nonblocking behavior is desired)
> c. the efficiency of the operation matters.
> Given the above three requirements, it seems a batching blocking API is
> fine?
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> 
> 
> On Thu, Feb 2, 2017 at 5:54 PM, Dong Lin  wrote:
> 
> > Hey Colin,
> >
> > Thanks for the KIP. I have a few comments below:
> >
> > - I share similar view with Ismael that a Future-based API is better.
> > PurgeDataFrom() is an example API that user may want to do it
> > asynchronously even though there is only one request in flight at a time.
> > In the future we may also have some admin operation that allows user to
> > move replica from one broker to another, which also needs to work in both
> > sync and async style. It seems more generic to return Future for any API
> > that requires both mode.
> >
> > - I am not sure if it is the cleanest way to have enum classes
> > CreateTopicsFlags and DeleteTopicsFlags. Are we going to create such class
> > for every future API that requires configuration? It may be more generic to
> > provide Map to any admin API that operates on multiple entries.
> > For example, deleteTopic(Map). And it can be Map > Properties> for those API that requires multiple configs per entry. And we
> > can provide default value, doc, config name for those API as we do
> > with AbstractConfig.
> >
> > - I not sure if "Try" is very intuitive to Java developer. Is there any
> > counterpart of scala's "Try" in java? We actually have quite a few existing
> > classes in Kafka that address the same problem, such as
> > ProduceRequestResult, LogAppendResult etc. Maybe we can follow the same
> > conversion and use *Result as this class name.
> >
> > - How are we going to allow user to specify timeout for blocking APIs such
> > as deleteTopic? Is this configured per AdminClient, or should it be
> > specified in the API parameter?
> >
> > - Are we going to have this class initiate its own thread, as we do with
> > Sender class, to send/receive requests? If yes, it will be useful to have
> > have a class that extends AbstractConfig and specifies config and their
> > default values.
> >
> > Thanks,
> > Dong
> >
> >
> >
> > On Thu, Feb 2, 2017 at 3:02 PM, Ismael Juma  wrote:
> >
> > > Hi Colin,
> > >
> > > Thanks for the KIP, great to make progress on this. I have some initial
> > > comments, will post more later:
> > >
> > > 1. We have KafkaProducer that implements the Producer interface and
> > > KafkaConsumer that implements the Consumer interface. Maybe we could have
> > > KafkaAdminClient that implements the AdminClient interface? Or maybe just
> > > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think
> > we
> > > should worry about a name clash with the internal AdminClient written in
> > > Scala. That will go away soon enough and choosing a good name for the
> > > public class is what matters.
> > >
> > > 2. We should include the proposed package name in the KIP
> > > (probably org.apache.kafka.clients.admin?).
> > >
> > > 3. It would be good to list the supported configs.
> > >
> > > 4. KIP-107, which passed the vote, specifies the introduction of a method
> > > to AdminClient with the following signature. We should figure out how it
> > > would look given this proposal.
> > >
> > > Future>
> > > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-03 Thread Colin McCabe
On Thu, Feb 2, 2017, at 17:54, Dong Lin wrote:
> Hey Colin,
> 
> Thanks for the KIP. I have a few comments below:
> 
> - I share similar view with Ismael that a Future-based API is better.
> PurgeDataFrom() is an example API that user may want to do it
> asynchronously even though there is only one request in flight at a time.
> In the future we may also have some admin operation that allows user to
> move replica from one broker to another, which also needs to work in both
> sync and async style. It seems more generic to return Future for any
> API
> that requires both mode.
> 
> - I am not sure if it is the cleanest way to have enum classes
> CreateTopicsFlags and DeleteTopicsFlags. Are we going to create such
> class
> for every future API that requires configuration? It may be more generic
> to
> provide Map to any admin API that operates on multiple
> entries.
> For example, deleteTopic(Map). And it can be Map Properties> for those API that requires multiple configs per entry. And
> we
> can provide default value, doc, config name for those API as we do
> with AbstractConfig.

Thanks for the comments, Dong L.

EnumSet, EnumSet, and so forth are
type-safe ways for the user to pass a set of boolean flags to the
function.  It is basically a replacement for having an api like
createTopics(, boolean nonblocking, boolean validateOnly).  It is
preferrable to having the boolean flags because it's immediately clear
when reading the code what createTopics(...,
CreateTopicsFlags.VALIDATE_ONLY) means, whereas it is not immediately
clear what createTopics(..., false, true) means.  It also prevents
having lots of function overloads over time, which becomes confusing for
users.  The EnumSet is not intended as a replacement for all possible
future arguments we might add, but just an easy way to add more boolean
arguments later without adding messy function overloads or type
signatures with many booleans.

Map is not type-safe, and I don't think we should use it in
our APIs.  Instead, we should just add function overloads if necessary.

> 
> - I not sure if "Try" is very intuitive to Java developer. Is there any
> counterpart of scala's "Try" in java

Unfortunately, there is no equivalent to Try in the standard Java
library.  That is the first place I checked, and I spent a while
searching.  The closest is probably Java 8's Optional.  However,
Optional just allows us to express Some(thing) or None, so callers would
not be able to determine what the error was.  

> We actually have quite a few
> existing
> classes in Kafka that address the same problem, such as
> ProduceRequestResult, LogAppendResult etc. Maybe we can follow the same
> conversion and use *Result as this class name.

Hmm.  ProduceRequestResult and LogAppendResult just store an exception
alongside the data, and make the caller responsible for checking whether
the exception is null (or None) before looking at the data.  I don't
think this is a good solution, because if the user forgets to do this,
they will interpret whatever is in the data fields as valid, even when
it is not.  ProduceRequestResult and LogAppendResult are also internal
classes, not user-facing APIs, so we did not spend time thinking about
how to make them easy to use for end-users.

> 
> - How are we going to allow user to specify timeout for blocking APIs
> such
> as deleteTopic? Is this configured per AdminClient, or should it be
> specified in the API parameter?

Right now, it is specified by the configuration of the AdminClient.

> 
> - Are we going to have this class initiate its own thread, as we do with
> Sender class, to send/receive requests? If yes, it will be useful to have
> have a class that extends AbstractConfig and specifies config and their
> default values.

Yes, I agree.  I will add this to the KIP.

best,
Colin

> 
> Thanks,
> Dong
> 
> 
> 
> On Thu, Feb 2, 2017 at 3:02 PM, Ismael Juma  wrote:
> 
> > Hi Colin,
> >
> > Thanks for the KIP, great to make progress on this. I have some initial
> > comments, will post more later:
> >
> > 1. We have KafkaProducer that implements the Producer interface and
> > KafkaConsumer that implements the Consumer interface. Maybe we could have
> > KafkaAdminClient that implements the AdminClient interface? Or maybe just
> > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think we
> > should worry about a name clash with the internal AdminClient written in
> > Scala. That will go away soon enough and choosing a good name for the
> > public class is what matters.
> >
> > 2. We should include the proposed package name in the KIP
> > (probably org.apache.kafka.clients.admin?).
> >
> > 3. It would be good to list the supported configs.
> >
> > 4. KIP-107, which passed the vote, specifies the introduction of a method
> > to AdminClient with the following signature. We should figure out how it
> > would look given this proposal.
> >
> > 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Becket Qin
Hi Colin,

Thanks for the KIP. An admin client is probably a must after we block
direct access to ZK. Some comments and thoughts below:

1. Do we have a clear scope for the admin client? It might be worth
thinking about the entire user experience of the admin client. Ideally we
may want to have a single client to do all the administrative work instead
of having multiple ways to do different things. For example, do we want to
add topic configurations change API in the admin client? What about
partition movements and preferred leader election? Those are also
administrative tasks which seem reasonable to be integrated into the admin
client.

2. Regarding the Future based async Ops v.s. batching of Ops, I would
prefer supporting batching if possible. That usually introduce much less
overhead when doing some big operations, e.g. in controller we have been
putting quite some efforts to batch the requests. For admin client, my
understanding is that the operations are:
a. rare and potentially big
b. likely OK to block (it would be good to see some use cases where a
nonblocking behavior is desired)
c. the efficiency of the operation matters.
Given the above three requirements, it seems a batching blocking API is
fine?

Thanks,

Jiangjie (Becket) Qin



On Thu, Feb 2, 2017 at 5:54 PM, Dong Lin  wrote:

> Hey Colin,
>
> Thanks for the KIP. I have a few comments below:
>
> - I share similar view with Ismael that a Future-based API is better.
> PurgeDataFrom() is an example API that user may want to do it
> asynchronously even though there is only one request in flight at a time.
> In the future we may also have some admin operation that allows user to
> move replica from one broker to another, which also needs to work in both
> sync and async style. It seems more generic to return Future for any API
> that requires both mode.
>
> - I am not sure if it is the cleanest way to have enum classes
> CreateTopicsFlags and DeleteTopicsFlags. Are we going to create such class
> for every future API that requires configuration? It may be more generic to
> provide Map to any admin API that operates on multiple entries.
> For example, deleteTopic(Map). And it can be Map Properties> for those API that requires multiple configs per entry. And we
> can provide default value, doc, config name for those API as we do
> with AbstractConfig.
>
> - I not sure if "Try" is very intuitive to Java developer. Is there any
> counterpart of scala's "Try" in java? We actually have quite a few existing
> classes in Kafka that address the same problem, such as
> ProduceRequestResult, LogAppendResult etc. Maybe we can follow the same
> conversion and use *Result as this class name.
>
> - How are we going to allow user to specify timeout for blocking APIs such
> as deleteTopic? Is this configured per AdminClient, or should it be
> specified in the API parameter?
>
> - Are we going to have this class initiate its own thread, as we do with
> Sender class, to send/receive requests? If yes, it will be useful to have
> have a class that extends AbstractConfig and specifies config and their
> default values.
>
> Thanks,
> Dong
>
>
>
> On Thu, Feb 2, 2017 at 3:02 PM, Ismael Juma  wrote:
>
> > Hi Colin,
> >
> > Thanks for the KIP, great to make progress on this. I have some initial
> > comments, will post more later:
> >
> > 1. We have KafkaProducer that implements the Producer interface and
> > KafkaConsumer that implements the Consumer interface. Maybe we could have
> > KafkaAdminClient that implements the AdminClient interface? Or maybe just
> > KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think
> we
> > should worry about a name clash with the internal AdminClient written in
> > Scala. That will go away soon enough and choosing a good name for the
> > public class is what matters.
> >
> > 2. We should include the proposed package name in the KIP
> > (probably org.apache.kafka.clients.admin?).
> >
> > 3. It would be good to list the supported configs.
> >
> > 4. KIP-107, which passed the vote, specifies the introduction of a method
> > to AdminClient with the following signature. We should figure out how it
> > would look given this proposal.
> >
> > Future>
> > purgeDataBefore(Map offsetForPartition)
> >
> > 5. I am not sure about rejecting the Futures-based API. I think I would
> > prefer that, personally. Grant had an interesting idea of not exposing
> the
> > batch methods in the AdminClient to start with to keep it simple and
> > relying on a Future based API to make it easy for users to run things
> > concurrently. This is consistent with the producer and Java 8 makes
> things
> > a lot nicer with CompletableFuture (similar to Scala Futures). I will
> think
> > more about this and other details of the proposal and send a follow-up.
> >
> > Ismael
> >
> > On Thu, Feb 2, 2017 at 6:54 PM, Colin 

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Dong Lin
Hey Colin,

Thanks for the KIP. I have a few comments below:

- I share similar view with Ismael that a Future-based API is better.
PurgeDataFrom() is an example API that user may want to do it
asynchronously even though there is only one request in flight at a time.
In the future we may also have some admin operation that allows user to
move replica from one broker to another, which also needs to work in both
sync and async style. It seems more generic to return Future for any API
that requires both mode.

- I am not sure if it is the cleanest way to have enum classes
CreateTopicsFlags and DeleteTopicsFlags. Are we going to create such class
for every future API that requires configuration? It may be more generic to
provide Map to any admin API that operates on multiple entries.
For example, deleteTopic(Map). And it can be Map for those API that requires multiple configs per entry. And we
can provide default value, doc, config name for those API as we do
with AbstractConfig.

- I not sure if "Try" is very intuitive to Java developer. Is there any
counterpart of scala's "Try" in java? We actually have quite a few existing
classes in Kafka that address the same problem, such as
ProduceRequestResult, LogAppendResult etc. Maybe we can follow the same
conversion and use *Result as this class name.

- How are we going to allow user to specify timeout for blocking APIs such
as deleteTopic? Is this configured per AdminClient, or should it be
specified in the API parameter?

- Are we going to have this class initiate its own thread, as we do with
Sender class, to send/receive requests? If yes, it will be useful to have
have a class that extends AbstractConfig and specifies config and their
default values.

Thanks,
Dong



On Thu, Feb 2, 2017 at 3:02 PM, Ismael Juma  wrote:

> Hi Colin,
>
> Thanks for the KIP, great to make progress on this. I have some initial
> comments, will post more later:
>
> 1. We have KafkaProducer that implements the Producer interface and
> KafkaConsumer that implements the Consumer interface. Maybe we could have
> KafkaAdminClient that implements the AdminClient interface? Or maybe just
> KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think we
> should worry about a name clash with the internal AdminClient written in
> Scala. That will go away soon enough and choosing a good name for the
> public class is what matters.
>
> 2. We should include the proposed package name in the KIP
> (probably org.apache.kafka.clients.admin?).
>
> 3. It would be good to list the supported configs.
>
> 4. KIP-107, which passed the vote, specifies the introduction of a method
> to AdminClient with the following signature. We should figure out how it
> would look given this proposal.
>
> Future>
> purgeDataBefore(Map offsetForPartition)
>
> 5. I am not sure about rejecting the Futures-based API. I think I would
> prefer that, personally. Grant had an interesting idea of not exposing the
> batch methods in the AdminClient to start with to keep it simple and
> relying on a Future based API to make it easy for users to run things
> concurrently. This is consistent with the producer and Java 8 makes things
> a lot nicer with CompletableFuture (similar to Scala Futures). I will think
> more about this and other details of the proposal and send a follow-up.
>
> Ismael
>
> On Thu, Feb 2, 2017 at 6:54 PM, Colin McCabe  wrote:
>
> > Hi all,
> >
> > I wrote up a Kafka improvement proposal for adding an
> > AdministrativeClient interface.  This is a continuation of the work on
> > KIP-4 towards centralized administrative operations.  Please check out
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%
> 3A+Add+a+public+
> > AdministrativeClient+API+for+Kafka+admin+operations
> >
> > regards,
> > Colin
> >
>


Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Ismael Juma
Hi Colin,

Thanks for the KIP, great to make progress on this. I have some initial
comments, will post more later:

1. We have KafkaProducer that implements the Producer interface and
KafkaConsumer that implements the Consumer interface. Maybe we could have
KafkaAdminClient that implements the AdminClient interface? Or maybe just
KafkaAdmin. Not sure, some ideas for consideration. Also, I don't think we
should worry about a name clash with the internal AdminClient written in
Scala. That will go away soon enough and choosing a good name for the
public class is what matters.

2. We should include the proposed package name in the KIP
(probably org.apache.kafka.clients.admin?).

3. It would be good to list the supported configs.

4. KIP-107, which passed the vote, specifies the introduction of a method
to AdminClient with the following signature. We should figure out how it
would look given this proposal.

Future>
purgeDataBefore(Map offsetForPartition)

5. I am not sure about rejecting the Futures-based API. I think I would
prefer that, personally. Grant had an interesting idea of not exposing the
batch methods in the AdminClient to start with to keep it simple and
relying on a Future based API to make it easy for users to run things
concurrently. This is consistent with the producer and Java 8 makes things
a lot nicer with CompletableFuture (similar to Scala Futures). I will think
more about this and other details of the proposal and send a follow-up.

Ismael

On Thu, Feb 2, 2017 at 6:54 PM, Colin McCabe  wrote:

> Hi all,
>
> I wrote up a Kafka improvement proposal for adding an
> AdministrativeClient interface.  This is a continuation of the work on
> KIP-4 towards centralized administrative operations.  Please check out
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public+
> AdministrativeClient+API+for+Kafka+admin+operations
>
> regards,
> Colin
>


[DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Colin McCabe
Hi all,

I wrote up a Kafka improvement proposal for adding an
AdministrativeClient interface.  This is a continuation of the work on
KIP-4 towards centralized administrative operations.  Please check out
https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public+AdministrativeClient+API+for+Kafka+admin+operations

regards,
Colin