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
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:
> >
>
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 consum
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 bat
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
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
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 si
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 t
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, bu
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 thin
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, w
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.
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
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 li
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
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 wi
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
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 ra
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
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
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
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 partit
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 c
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
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
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 co
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 f
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. offset
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
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
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 ag
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 num
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(...)
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
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 r
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
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
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 s
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
o
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 ov
+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
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 Adm
>>- 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 th
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 r
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
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
in
>
>
> Regards,
>
>
> -- huxi
>
>
>
> 发件人: radai
> 发送时间: 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
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 informat
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, G
+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 discussio
+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 ke
+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 th
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 t
-- huxi
>
>
> ____________
> 发件人: radai
> 发送时间: 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 storag
@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, F
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 li
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 wrot
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 t
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,
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
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.Topic
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
> th
+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
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 operation
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
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
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 interfac
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
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
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 s
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
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 Admi
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
73 matches
Mail list logo